diff --git a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java index fc8689354274f..ecbb0ccb4d200 100644 --- a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java +++ b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java @@ -19,6 +19,7 @@ import java.io.IOException; +import org.apache.spark.errors.SparkCoreErrors; import org.apache.spark.unsafe.array.LongArray; import org.apache.spark.unsafe.memory.MemoryBlock; @@ -153,9 +154,6 @@ private void throwOom(final MemoryBlock page, final long required) { taskMemoryManager.freePage(page, this); } taskMemoryManager.showMemoryUsage(); - // checkstyle.off: RegexpSinglelineJava - throw new SparkOutOfMemoryError("UNABLE_TO_ACQUIRE_MEMORY", - new String[]{Long.toString(required), Long.toString(got)}); - // checkstyle.on: RegexpSinglelineJava + throw SparkCoreErrors.outOfMemoryError(required, got); } } diff --git a/core/src/main/java/org/apache/spark/memory/SparkOutOfMemoryError.java b/core/src/main/java/org/apache/spark/memory/SparkOutOfMemoryError.java index 3ff3094456ef1..cfbf2e574787d 100644 --- a/core/src/main/java/org/apache/spark/memory/SparkOutOfMemoryError.java +++ b/core/src/main/java/org/apache/spark/memory/SparkOutOfMemoryError.java @@ -20,6 +20,8 @@ import org.apache.spark.SparkThrowableHelper; import org.apache.spark.annotation.Private; +import java.util.Map; + /** * This exception is thrown when a task can not acquire memory from the Memory manager. * Instead of throwing {@link OutOfMemoryError}, which kills the executor, @@ -28,7 +30,7 @@ @Private public final class SparkOutOfMemoryError extends OutOfMemoryError implements SparkThrowable { String errorClass; - String[] messageParameters; + Map messageParameters; public SparkOutOfMemoryError(String s) { super(s); @@ -38,7 +40,7 @@ public SparkOutOfMemoryError(OutOfMemoryError e) { super(e.getMessage()); } - public SparkOutOfMemoryError(String errorClass, String[] messageParameters) { + public SparkOutOfMemoryError(String errorClass, Map messageParameters) { super(SparkThrowableHelper.getMessage(errorClass, null, messageParameters)); this.errorClass = errorClass; this.messageParameters = messageParameters; @@ -46,7 +48,7 @@ public SparkOutOfMemoryError(String errorClass, String[] messageParameters) { @Override public String[] getMessageParameters() { - return messageParameters; + return SparkThrowableHelper.getMessageParameters(errorClass, null, messageParameters); } @Override diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 29f1f4f0b3007..999a9b0a4aec6 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -423,6 +423,11 @@ ], "sqlState" : "22023" }, + "TOO_MANY_ARRAY_ELEMENTS" : { + "message" : [ + "Cannot initialize array with elements of size " + ] + }, "UNABLE_TO_ACQUIRE_MEMORY" : { "message" : [ "Unable to acquire bytes of memory, got " @@ -484,7 +489,7 @@ }, "UNRESOLVED_MAP_KEY" : { "message" : [ - "Cannot resolve column as a map key. If the key is a string literal, please add single quotes around it." + "Cannot resolve column as a map key. If the key is a string literal, please add single quotes around it." ], "subClass" : { "WITHOUT_SUGGESTION" : { diff --git a/core/src/main/scala/org/apache/spark/SparkException.scala b/core/src/main/scala/org/apache/spark/SparkException.scala index aea796685ee4b..dba6ef9347ff0 100644 --- a/core/src/main/scala/org/apache/spark/SparkException.scala +++ b/core/src/main/scala/org/apache/spark/SparkException.scala @@ -17,7 +17,7 @@ package org.apache.spark -import java.io.{FileNotFoundException, IOException} +import java.io.FileNotFoundException import java.sql.{SQLException, SQLFeatureNotSupportedException} import java.time.DateTimeException import java.util.ConcurrentModificationException @@ -29,7 +29,7 @@ class SparkException( cause: Throwable, errorClass: Option[String], errorSubClass: Option[String], - messageParameters: Array[String], + messageParameters: Map[String, String], context: Array[QueryContext] = Array.empty) extends Exception(message, cause) with SparkThrowable { @@ -37,23 +37,24 @@ class SparkException( message: String, cause: Throwable, errorClass: Option[String], - messageParameters: Array[String]) = - this(message = message, - cause = cause, - errorClass = errorClass, - errorSubClass = None, - messageParameters = messageParameters) + messageParameters: Map[String, String]) = + this( + message = message, + cause = cause, + errorClass = errorClass, + errorSubClass = None, + messageParameters = messageParameters) def this(message: String, cause: Throwable) = this(message = message, cause = cause, errorClass = None, errorSubClass = None, - messageParameters = Array.empty) + messageParameters = Map.empty) def this(message: String) = this(message = message, cause = null) def this( errorClass: String, - messageParameters: Array[String], + messageParameters: Map[String, String], cause: Throwable, context: Array[QueryContext], summary: String) = @@ -65,7 +66,7 @@ class SparkException( messageParameters = messageParameters, context) - def this(errorClass: String, messageParameters: Array[String], cause: Throwable) = + def this(errorClass: String, messageParameters: Map[String, String], cause: Throwable) = this( message = SparkThrowableHelper.getMessage(errorClass, null, messageParameters), cause = cause, @@ -76,7 +77,7 @@ class SparkException( def this( errorClass: String, errorSubClass: String, - messageParameters: Array[String], + messageParameters: Map[String, String], cause: Throwable) = this( message = SparkThrowableHelper.getMessage(errorClass, errorSubClass, messageParameters), @@ -85,7 +86,12 @@ class SparkException( errorSubClass = Some(errorSubClass), messageParameters = messageParameters) - override def getMessageParameters: Array[String] = messageParameters + override def getMessageParameters: Array[String] = { + errorClass.map { ec => + SparkThrowableHelper.getMessageParameters(ec, errorSubClass.orNull, messageParameters) + }.getOrElse(Array.empty) + } + override def getErrorClass: String = errorClass.orNull override def getErrorSubClass: String = errorSubClass.orNull override def getQueryContext: Array[QueryContext] = context @@ -93,11 +99,17 @@ class SparkException( object SparkException { def internalError(msg: String): SparkException = { - new SparkException(errorClass = "INTERNAL_ERROR", messageParameters = Array(msg), cause = null) + new SparkException( + errorClass = "INTERNAL_ERROR", + messageParameters = Map("message" -> msg), + cause = null) } def internalError(msg: String, cause: Throwable): SparkException = { - new SparkException(errorClass = "INTERNAL_ERROR", messageParameters = Array(msg), cause = cause) + new SparkException( + errorClass = "INTERNAL_ERROR", + messageParameters = Map("message" -> msg), + cause = cause) } } @@ -128,13 +140,16 @@ private[spark] case class ExecutorDeadException(message: String) private[spark] class SparkUpgradeException( errorClass: String, errorSubClass: Option[String] = None, - messageParameters: Array[String], + messageParameters: Map[String, String], cause: Throwable) - extends RuntimeException(SparkThrowableHelper.getMessage(errorClass, errorSubClass.orNull, - messageParameters), cause) - with SparkThrowable { + extends RuntimeException( + SparkThrowableHelper.getMessage(errorClass, errorSubClass.orNull, messageParameters), cause) + with SparkThrowable { + + override def getMessageParameters: Array[String] = { + SparkThrowableHelper.getMessageParameters(errorClass, errorSubClass.orNull, messageParameters) + } - override def getMessageParameters: Array[String] = messageParameters override def getErrorClass: String = errorClass override def getErrorSubClass: String = errorSubClass.orNull} @@ -144,14 +159,17 @@ private[spark] class SparkUpgradeException( private[spark] class SparkArithmeticException( errorClass: String, errorSubClass: Option[String] = None, - messageParameters: Array[String], + messageParameters: Map[String, String], context: Array[QueryContext], summary: String) extends ArithmeticException( SparkThrowableHelper.getMessage(errorClass, errorSubClass.orNull, messageParameters, summary)) - with SparkThrowable { + with SparkThrowable { + + override def getMessageParameters: Array[String] = { + SparkThrowableHelper.getMessageParameters(errorClass, errorSubClass.orNull, messageParameters) + } - override def getMessageParameters: Array[String] = messageParameters override def getErrorClass: String = errorClass override def getErrorSubClass: String = errorSubClass.orNull override def getQueryContext: Array[QueryContext] = context @@ -163,21 +181,24 @@ private[spark] class SparkArithmeticException( private[spark] class SparkUnsupportedOperationException( errorClass: String, errorSubClass: Option[String] = None, - messageParameters: Array[String]) + messageParameters: Map[String, String]) extends UnsupportedOperationException( SparkThrowableHelper.getMessage(errorClass, errorSubClass.orNull, messageParameters)) - with SparkThrowable { + with SparkThrowable { def this( errorClass: String, errorSubClass: String, - messageParameters: Array[String]) = + messageParameters: Map[String, String]) = this( errorClass = errorClass, errorSubClass = Some(errorSubClass), messageParameters = messageParameters) - override def getMessageParameters: Array[String] = messageParameters + override def getMessageParameters: Array[String] = { + SparkThrowableHelper.getMessageParameters(errorClass, errorSubClass.orNull, messageParameters) + } + override def getErrorClass: String = errorClass override def getErrorSubClass: String = errorSubClass.orNull } @@ -188,13 +209,16 @@ private[spark] class SparkUnsupportedOperationException( private[spark] class SparkClassNotFoundException( errorClass: String, errorSubClass: Option[String] = None, - messageParameters: Array[String], + messageParameters: Map[String, String], cause: Throwable = null) extends ClassNotFoundException( SparkThrowableHelper.getMessage(errorClass, errorSubClass.orNull, messageParameters), cause) - with SparkThrowable { + with SparkThrowable { + + override def getMessageParameters: Array[String] = { + SparkThrowableHelper.getMessageParameters(errorClass, errorSubClass.orNull, messageParameters) + } - override def getMessageParameters: Array[String] = messageParameters override def getErrorClass: String = errorClass override def getErrorSubClass: String = errorSubClass.orNull} @@ -204,13 +228,16 @@ private[spark] class SparkClassNotFoundException( private[spark] class SparkConcurrentModificationException( errorClass: String, errorSubClass: Option[String] = None, - messageParameters: Array[String], + messageParameters: Map[String, String], cause: Throwable = null) extends ConcurrentModificationException( SparkThrowableHelper.getMessage(errorClass, errorSubClass.orNull, messageParameters), cause) - with SparkThrowable { + with SparkThrowable { + + override def getMessageParameters: Array[String] = { + SparkThrowableHelper.getMessageParameters(errorClass, errorSubClass.orNull, messageParameters) + } - override def getMessageParameters: Array[String] = messageParameters override def getErrorClass: String = errorClass override def getErrorSubClass: String = errorSubClass.orNull} @@ -220,14 +247,17 @@ private[spark] class SparkConcurrentModificationException( private[spark] class SparkDateTimeException( errorClass: String, errorSubClass: Option[String] = None, - messageParameters: Array[String], + messageParameters: Map[String, String], context: Array[QueryContext], summary: String) extends DateTimeException( SparkThrowableHelper.getMessage(errorClass, errorSubClass.orNull, messageParameters, summary)) - with SparkThrowable { + with SparkThrowable { + + override def getMessageParameters: Array[String] = { + SparkThrowableHelper.getMessageParameters(errorClass, errorSubClass.orNull, messageParameters) + } - override def getMessageParameters: Array[String] = messageParameters override def getErrorClass: String = errorClass override def getErrorSubClass: String = errorSubClass.orNull override def getQueryContext: Array[QueryContext] = context @@ -239,12 +269,15 @@ private[spark] class SparkDateTimeException( private[spark] class SparkFileAlreadyExistsException( errorClass: String, errorSubClass: Option[String] = None, - messageParameters: Array[String]) + messageParameters: Map[String, String]) extends FileAlreadyExistsException( SparkThrowableHelper.getMessage(errorClass, errorSubClass.orNull, messageParameters)) - with SparkThrowable { + with SparkThrowable { + + override def getMessageParameters: Array[String] = { + SparkThrowableHelper.getMessageParameters(errorClass, errorSubClass.orNull, messageParameters) + } - override def getMessageParameters: Array[String] = messageParameters override def getErrorClass: String = errorClass override def getErrorSubClass: String = errorSubClass.orNull} @@ -254,12 +287,15 @@ private[spark] class SparkFileAlreadyExistsException( private[spark] class SparkFileNotFoundException( errorClass: String, errorSubClass: Option[String] = None, - messageParameters: Array[String]) + messageParameters: Map[String, String]) extends FileNotFoundException( SparkThrowableHelper.getMessage(errorClass, errorSubClass.orNull, messageParameters)) - with SparkThrowable { + with SparkThrowable { + + override def getMessageParameters: Array[String] = { + SparkThrowableHelper.getMessageParameters(errorClass, errorSubClass.orNull, messageParameters) + } - override def getMessageParameters: Array[String] = messageParameters override def getErrorClass: String = errorClass override def getErrorSubClass: String = errorSubClass.orNull} @@ -269,102 +305,84 @@ private[spark] class SparkFileNotFoundException( private[spark] class SparkNumberFormatException( errorClass: String, errorSubClass: Option[String] = None, - messageParameters: Array[String], + messageParameters: Map[String, String], context: Array[QueryContext], summary: String) extends NumberFormatException( SparkThrowableHelper.getMessage(errorClass, errorSubClass.orNull, messageParameters, summary)) - with SparkThrowable { + with SparkThrowable { + + override def getMessageParameters: Array[String] = { + SparkThrowableHelper.getMessageParameters(errorClass, errorSubClass.orNull, messageParameters) + } - override def getMessageParameters: Array[String] = messageParameters override def getErrorClass: String = errorClass override def getErrorSubClass: String = errorSubClass.orNull override def getQueryContext: Array[QueryContext] = context } -/** - * No such method exception thrown from Spark with an error class. - */ -private[spark] class SparkNoSuchMethodException( - errorClass: String, - errorSubClass: Option[String] = None, - messageParameters: Array[String]) - extends NoSuchMethodException( - SparkThrowableHelper.getMessage(errorClass, errorSubClass.orNull, messageParameters)) - with SparkThrowable { - - override def getMessageParameters: Array[String] = messageParameters - override def getErrorClass: String = errorClass - override def getErrorSubClass: String = errorSubClass.orNull} - /** * Illegal argument exception thrown from Spark with an error class. */ private[spark] class SparkIllegalArgumentException( errorClass: String, errorSubClass: Option[String] = None, - messageParameters: Array[String], + messageParameters: Map[String, String], context: Array[QueryContext] = Array.empty, summary: String = "") extends IllegalArgumentException( SparkThrowableHelper.getMessage(errorClass, errorSubClass.orNull, messageParameters, summary)) - with SparkThrowable { - - override def getMessageParameters: Array[String] = messageParameters - override def getErrorClass: String = errorClass - override def getErrorSubClass: String = errorSubClass.orNull - override def getQueryContext: Array[QueryContext] = context -} + with SparkThrowable { -/** - * IO exception thrown from Spark with an error class. - */ -private[spark] class SparkIOException( - errorClass: String, - errorSubClass: Option[String] = None, - messageParameters: Array[String]) - extends IOException( - SparkThrowableHelper.getMessage(errorClass, errorSubClass.orNull, messageParameters)) - with SparkThrowable { + override def getMessageParameters: Array[String] = { + SparkThrowableHelper.getMessageParameters(errorClass, errorSubClass.orNull, messageParameters) + } - override def getMessageParameters: Array[String] = messageParameters override def getErrorClass: String = errorClass override def getErrorSubClass: String = errorSubClass.orNull + override def getQueryContext: Array[QueryContext] = context } private[spark] class SparkRuntimeException( errorClass: String, errorSubClass: Option[String] = None, - messageParameters: Array[String], + messageParameters: Map[String, String], cause: Throwable = null, context: Array[QueryContext] = Array.empty, summary: String = "") extends RuntimeException( SparkThrowableHelper.getMessage(errorClass, errorSubClass.orNull, messageParameters, summary), cause) - with SparkThrowable { - - def this(errorClass: String, - errorSubClass: String, - messageParameters: Array[String], - cause: Throwable, - context: Array[QueryContext]) - = this(errorClass = errorClass, - errorSubClass = Some(errorSubClass), - messageParameters = messageParameters, - cause = cause, - context = context) - - def this(errorClass: String, - errorSubClass: String, - messageParameters: Array[String]) - = this(errorClass = errorClass, - errorSubClass = Some(errorSubClass), - messageParameters = messageParameters, - cause = null, - context = Array.empty[QueryContext]) - - override def getMessageParameters: Array[String] = messageParameters + with SparkThrowable { + + def this( + errorClass: String, + errorSubClass: String, + messageParameters: Map[String, String], + cause: Throwable, + context: Array[QueryContext]) + = this( + errorClass = errorClass, + errorSubClass = Some(errorSubClass), + messageParameters = messageParameters, + cause = cause, + context = context) + + def this( + errorClass: String, + errorSubClass: String, + messageParameters: Map[String, String]) + = this( + errorClass = errorClass, + errorSubClass = Some(errorSubClass), + messageParameters = messageParameters, + cause = null, + context = Array.empty[QueryContext]) + + override def getMessageParameters: Array[String] = { + SparkThrowableHelper.getMessageParameters(errorClass, errorSubClass.orNull, messageParameters) + } + override def getErrorClass: String = errorClass override def getErrorSubClass: String = errorSubClass.orNull override def getQueryContext: Array[QueryContext] = context @@ -376,12 +394,15 @@ private[spark] class SparkRuntimeException( private[spark] class SparkSecurityException( errorClass: String, errorSubClass: Option[String] = None, - messageParameters: Array[String]) + messageParameters: Map[String, String]) extends SecurityException( SparkThrowableHelper.getMessage(errorClass, errorSubClass.orNull, messageParameters)) - with SparkThrowable { + with SparkThrowable { + + override def getMessageParameters: Array[String] = { + SparkThrowableHelper.getMessageParameters(errorClass, errorSubClass.orNull, messageParameters) + } - override def getMessageParameters: Array[String] = messageParameters override def getErrorClass: String = errorClass override def getErrorSubClass: String = errorSubClass.orNull } @@ -392,14 +413,17 @@ private[spark] class SparkSecurityException( private[spark] class SparkArrayIndexOutOfBoundsException( errorClass: String, errorSubClass: Option[String] = None, - messageParameters: Array[String], + messageParameters: Map[String, String], context: Array[QueryContext], summary: String) extends ArrayIndexOutOfBoundsException( SparkThrowableHelper.getMessage(errorClass, errorSubClass.orNull, messageParameters, summary)) - with SparkThrowable { + with SparkThrowable { + + override def getMessageParameters: Array[String] = { + SparkThrowableHelper.getMessageParameters(errorClass, errorSubClass.orNull, messageParameters) + } - override def getMessageParameters: Array[String] = messageParameters override def getErrorClass: String = errorClass override def getErrorSubClass: String = errorSubClass.orNull override def getQueryContext: Array[QueryContext] = context @@ -410,40 +434,24 @@ private[spark] class SparkArrayIndexOutOfBoundsException( */ private[spark] class SparkSQLException( errorClass: String, - errorSubClass: Option[String] = None, - messageParameters: Array[String]) + errorSubClass: Option[String], + messageParameters: Map[String, String]) extends SQLException( SparkThrowableHelper.getMessage(errorClass, errorSubClass.orNull, messageParameters)) - with SparkThrowable { + with SparkThrowable { - def this(errorClass: String, messageParameters: Array[String]) = + def this(errorClass: String, messageParameters: Map[String, String]) = this( errorClass = errorClass, errorSubClass = None, messageParameters = messageParameters) - override def getMessageParameters: Array[String] = messageParameters - override def getErrorClass: String = errorClass - override def getErrorSubClass: String = errorSubClass.orNull -} - -/** - * No such element exception thrown from Spark with an error class. - */ -private[spark] class SparkNoSuchElementException( - errorClass: String, - errorSubClass: Option[String] = None, - messageParameters: Array[String], - context: Array[QueryContext], - summary: String) - extends NoSuchElementException( - SparkThrowableHelper.getMessage(errorClass, errorSubClass.orNull, messageParameters, summary)) - with SparkThrowable { + override def getMessageParameters: Array[String] = { + SparkThrowableHelper.getMessageParameters(errorClass, errorSubClass.orNull, messageParameters) + } - override def getMessageParameters: Array[String] = messageParameters override def getErrorClass: String = errorClass override def getErrorSubClass: String = errorSubClass.orNull - override def getQueryContext: Array[QueryContext] = context } /** @@ -452,19 +460,24 @@ private[spark] class SparkNoSuchElementException( private[spark] class SparkSQLFeatureNotSupportedException( errorClass: String, errorSubClass: Option[String] = None, - messageParameters: Array[String]) + messageParameters: Map[String, String]) extends SQLFeatureNotSupportedException( SparkThrowableHelper.getMessage(errorClass, errorSubClass.orNull, messageParameters)) - with SparkThrowable { + with SparkThrowable { - def this(errorClass: String, - errorSubClass: String, - messageParameters: Array[String]) = - this(errorClass = errorClass, + def this( + errorClass: String, + errorSubClass: String, + messageParameters: Map[String, String]) = + this( + errorClass = errorClass, errorSubClass = Some(errorSubClass), messageParameters = messageParameters) - override def getMessageParameters: Array[String] = messageParameters + override def getMessageParameters: Array[String] = { + SparkThrowableHelper.getMessageParameters(errorClass, errorSubClass.orNull, messageParameters) + } + override def getErrorClass: String = errorClass override def getErrorSubClass: String = errorSubClass.orNull } diff --git a/core/src/main/scala/org/apache/spark/SparkThrowableHelper.scala b/core/src/main/scala/org/apache/spark/SparkThrowableHelper.scala index 92128f2b48b87..81c5c6cb043b3 100644 --- a/core/src/main/scala/org/apache/spark/SparkThrowableHelper.scala +++ b/core/src/main/scala/org/apache/spark/SparkThrowableHelper.scala @@ -19,12 +19,14 @@ package org.apache.spark import java.net.URL +import scala.collection.JavaConverters._ import scala.collection.immutable.SortedMap import com.fasterxml.jackson.annotation.JsonIgnore import com.fasterxml.jackson.core.`type`.TypeReference import com.fasterxml.jackson.databind.json.JsonMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule +import org.apache.commons.text.StringSubstitutor import org.apache.spark.util.JsonProtocol.toJsonString import org.apache.spark.util.Utils @@ -76,22 +78,66 @@ private[spark] object SparkThrowableHelper { mapper.readValue(errorClassesUrl, new TypeReference[SortedMap[String, ErrorInfo]]() {}) } + def getParameterNames(errorClass: String, errorSubCLass: String): Array[String] = { + val errorInfo = errorClassToInfoMap.getOrElse(errorClass, + throw new IllegalArgumentException(s"Cannot find error class '$errorClass'")) + if (errorInfo.subClass.isEmpty && errorSubCLass != null) { + throw new IllegalArgumentException(s"'$errorClass' has no subclass") + } + if (errorInfo.subClass.isDefined && errorSubCLass == null) { + throw new IllegalArgumentException(s"'$errorClass' requires subclass") + } + var parameterizedMessage = errorInfo.messageFormat + if (errorInfo.subClass.isDefined) { + val givenSubClass = errorSubCLass + val errorSubInfo = errorInfo.subClass.get.getOrElse(givenSubClass, + throw new IllegalArgumentException(s"Cannot find sub error class '$givenSubClass'")) + parameterizedMessage = parameterizedMessage + errorSubInfo.messageFormat + } + val pattern = "<[a-zA-Z0-9_-]+>".r + val matches = pattern.findAllIn(parameterizedMessage) + val parameterSeq = matches.toArray + val parameterNames = parameterSeq.map(p => p.stripPrefix("<").stripSuffix(">")) + parameterNames + } + + def getMessageParameters( + errorClass: String, + errorSubCLass: String, + params: Map[String, String]): Array[String] = { + getParameterNames(errorClass, errorSubCLass).map(params.getOrElse(_, "?")) + } + + def getMessageParameters( + errorClass: String, + errorSubCLass: String, + params: java.util.Map[String, String]): Array[String] = { + getParameterNames(errorClass, errorSubCLass).map(params.getOrDefault(_, "?")) + } + def getMessage( errorClass: String, errorSubClass: String, - messageParameters: Array[String]): String = { + messageParameters: Map[String, String]): String = { getMessage(errorClass, errorSubClass, messageParameters, "") } def getMessage( errorClass: String, errorSubClass: String, - messageParameters: Array[String], + messageParameters: java.util.Map[String, String]): String = { + getMessage(errorClass, errorSubClass, messageParameters.asScala.toMap, "") + } + + def getMessage( + errorClass: String, + errorSubClass: String, + messageParameters: Map[String, String], context: String): String = { val errorInfo = errorClassToInfoMap.getOrElse(errorClass, throw new IllegalArgumentException(s"Cannot find error class '$errorClass'")) - val (displayClass, displayMessageParameters, displayFormat) = if (errorInfo.subClass.isEmpty) { - (errorClass, messageParameters, errorInfo.messageFormat) + val (displayClass, displayFormat) = if (errorInfo.subClass.isEmpty) { + (errorClass, errorInfo.messageFormat) } else { val subClasses = errorInfo.subClass.get if (errorSubClass == null) { @@ -99,40 +145,22 @@ private[spark] object SparkThrowableHelper { } val errorSubInfo = subClasses.getOrElse(errorSubClass, throw new IllegalArgumentException(s"Cannot find sub error class '$errorSubClass'")) - (errorClass + "." + errorSubClass, messageParameters, + (errorClass + "." + errorSubClass, errorInfo.messageFormat + " " + errorSubInfo.messageFormat) } - val displayMessage = String.format( - displayFormat.replaceAll("<[a-zA-Z0-9_-]+>", "%s"), - displayMessageParameters : _*) + val sub = new StringSubstitutor(messageParameters.asJava) + sub.setEnableUndefinedVariableException(true) + val displayMessage = try { + sub.replace(displayFormat.replaceAll("<([a-zA-Z0-9_-]+)>", "\\$\\{$1\\}")) + } catch { + case _: IllegalArgumentException => throw SparkException.internalError( + s"Undefined an error message parameter: $messageParameters") + } val displayQueryContext = (if (context.isEmpty) "" else "\n") + context s"[$displayClass] $displayMessage$displayQueryContext" } - def getParameterNames(errorClass: String, errorSubCLass: String): Array[String] = { - val errorInfo = errorClassToInfoMap.getOrElse(errorClass, - throw new IllegalArgumentException(s"Cannot find error class '$errorClass'")) - if (errorInfo.subClass.isEmpty && errorSubCLass != null) { - throw new IllegalArgumentException(s"'$errorClass' has no subclass") - } - if (errorInfo.subClass.isDefined && errorSubCLass == null) { - throw new IllegalArgumentException(s"'$errorClass' requires subclass") - } - var parameterizedMessage = errorInfo.messageFormat - if (errorInfo.subClass.isDefined) { - val givenSubClass = errorSubCLass - val errorSubInfo = errorInfo.subClass.get.getOrElse(givenSubClass, - throw new IllegalArgumentException(s"Cannot find sub error class '$givenSubClass'")) - parameterizedMessage = parameterizedMessage + errorSubInfo.messageFormat - } - val pattern = "<[a-zA-Z0-9_-]+>".r - val matches = pattern.findAllIn(parameterizedMessage) - val parameterSeq = matches.toArray - val parameterNames = parameterSeq.map(p => p.stripPrefix("<").stripSuffix(">")) - parameterNames - } - def getSqlState(errorClass: String): String = { Option(errorClass).flatMap(errorClassToInfoMap.get).flatMap(_.sqlState).orNull } @@ -175,7 +203,7 @@ private[spark] object SparkThrowableHelper { val parameterNames = e.getParameterNames if (!parameterNames.isEmpty) { g.writeObjectFieldStart("messageParameters") - (parameterNames zip e.getMessageParameters).foreach { case (name, value) => + (parameterNames zip e.getMessageParameters).sortBy(_._1).foreach { case (name, value) => g.writeStringField(name, value) } g.writeEndObject() diff --git a/core/src/main/scala/org/apache/spark/errors/SparkCoreErrors.scala b/core/src/main/scala/org/apache/spark/errors/SparkCoreErrors.scala index 1da0288446233..308ee003d5c2c 100644 --- a/core/src/main/scala/org/apache/spark/errors/SparkCoreErrors.scala +++ b/core/src/main/scala/org/apache/spark/errors/SparkCoreErrors.scala @@ -20,9 +20,12 @@ package org.apache.spark.errors import java.io.IOException import java.util.concurrent.TimeoutException +import scala.collection.JavaConverters._ + import org.apache.hadoop.fs.Path import org.apache.spark.{SparkException, TaskNotSerializableException} +import org.apache.spark.memory.SparkOutOfMemoryError import org.apache.spark.scheduler.{BarrierJobRunWithDynamicAllocationException, BarrierJobSlotsNumberCheckFailed, BarrierJobUnsupportedRDDChainException} import org.apache.spark.shuffle.{FetchFailedException, ShuffleManager} import org.apache.spark.storage.{BlockId, BlockManagerId, BlockNotFoundException, BlockSavedOnDecommissionedBlockManagerException, RDDBlockId, UnrecognizedBlockId} @@ -317,12 +320,24 @@ private[spark] object SparkCoreErrors { } def graphiteSinkInvalidProtocolError(invalidProtocol: String): Throwable = { - new SparkException(errorClass = "GRAPHITE_SINK_INVALID_PROTOCOL", - messageParameters = Array(invalidProtocol), cause = null) + new SparkException( + errorClass = "GRAPHITE_SINK_INVALID_PROTOCOL", + messageParameters = Map("protocol" -> invalidProtocol), + cause = null) } def graphiteSinkPropertyMissingError(missingProperty: String): Throwable = { - new SparkException(errorClass = "GRAPHITE_SINK_PROPERTY_MISSING", - messageParameters = Array(missingProperty), cause = null) + new SparkException( + errorClass = "GRAPHITE_SINK_PROPERTY_MISSING", + messageParameters = Map("property" -> missingProperty), + cause = null) + } + + def outOfMemoryError(requestedBytes: Long, receivedBytes: Long): OutOfMemoryError = { + new SparkOutOfMemoryError( + "UNABLE_TO_ACQUIRE_MEMORY", + Map( + "requestedBytes" -> requestedBytes.toString, + "receivedBytes" -> receivedBytes.toString).asJava) } } diff --git a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala index dda86387f156f..cbf273dc5c857 100644 --- a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark import java.io.File import java.nio.charset.StandardCharsets import java.nio.file.Files -import java.util.IllegalFormatException import com.fasterxml.jackson.annotation.JsonInclude.Include import com.fasterxml.jackson.core.JsonParser.Feature.STRICT_DUPLICATE_DETECTION @@ -146,24 +145,26 @@ class SparkThrowableSuite extends SparkFunSuite { test("Check if error class is missing") { val ex1 = intercept[IllegalArgumentException] { - getMessage("", null, Array.empty) + getMessage("", null, Map.empty[String, String]) } assert(ex1.getMessage == "Cannot find error class ''") val ex2 = intercept[IllegalArgumentException] { - getMessage("LOREM_IPSUM", null, Array.empty) + getMessage("LOREM_IPSUM", null, Map.empty[String, String]) } assert(ex2.getMessage == "Cannot find error class 'LOREM_IPSUM'") } test("Check if message parameters match message format") { // Requires 2 args - intercept[IllegalFormatException] { - getMessage("UNRESOLVED_COLUMN", "WITHOUT_SUGGESTION", Array.empty) + val e = intercept[SparkException] { + getMessage("UNRESOLVED_COLUMN", "WITHOUT_SUGGESTION", Map.empty[String, String]) } + assert(e.getErrorClass === "INTERNAL_ERROR") + assert(e.getMessageParameters.head.contains("Undefined an error message parameter")) // Does not fail with too many args (expects 0 args) - assert(getMessage("DIVIDE_BY_ZERO", null, Array("foo", "bar", "baz")) == + assert(getMessage("DIVIDE_BY_ZERO", null, Map("config" -> "foo", "a" -> "bar")) == "[DIVIDE_BY_ZERO] Division by zero. " + "Use `try_divide` to tolerate divisor being 0 and return NULL instead. " + "If necessary set foo to \"false\" " + @@ -175,7 +176,20 @@ class SparkThrowableSuite extends SparkFunSuite { getMessage( "UNRESOLVED_COLUMN", "WITH_SUGGESTION", - Array("`foo`", "`bar`, `baz`") + Map("objectName" -> "`foo`", "proposal" -> "`bar`, `baz`") + ) == + "[UNRESOLVED_COLUMN.WITH_SUGGESTION] A column or function parameter with " + + "name `foo` cannot be resolved. Did you mean one of the following? [`bar`, `baz`]" + ) + + assert( + getMessage( + "UNRESOLVED_COLUMN", + "WITH_SUGGESTION", + Map( + "objectName" -> "`foo`", + "proposal" -> "`bar`, `baz`"), + "" ) == "[UNRESOLVED_COLUMN.WITH_SUGGESTION] A column or function parameter with " + "name `foo` cannot be resolved. Did you mean one of the following? [`bar`, `baz`]" @@ -199,7 +213,7 @@ class SparkThrowableSuite extends SparkFunSuite { try { throw new SparkException( errorClass = "WRITING_JOB_ABORTED", - messageParameters = Array.empty, + messageParameters = Map.empty, cause = null) } catch { case e: SparkThrowable => @@ -215,7 +229,7 @@ class SparkThrowableSuite extends SparkFunSuite { try { throw new SparkException( errorClass = "INTERNAL_ERROR", - messageParameters = Array("this is an internal error"), + messageParameters = Map("message" -> "this is an internal error"), cause = null ) } catch { @@ -240,7 +254,7 @@ class SparkThrowableSuite extends SparkFunSuite { val e = new SparkArithmeticException( errorClass = "DIVIDE_BY_ZERO", errorSubClass = None, - messageParameters = Array("CONFIG"), + messageParameters = Map("config" -> "CONFIG"), context = Array(new TestQueryContext), summary = "Query summary") @@ -283,7 +297,7 @@ class SparkThrowableSuite extends SparkFunSuite { val e2 = new SparkIllegalArgumentException( errorClass = "UNSUPPORTED_SAVE_MODE", errorSubClass = Some("EXISTENT_PATH"), - messageParameters = Array("UNSUPPORTED_MODE")) + messageParameters = Map("saveMode" -> "UNSUPPORTED_MODE")) assert(SparkThrowableHelper.getMessage(e2, STANDARD) === """{ | "errorClass" : "UNSUPPORTED_SAVE_MODE", diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala index a810688a50320..5b6fb31d598ac 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala @@ -107,12 +107,11 @@ class ShuffleExternalSorterSuite extends SparkFunSuite with LocalSparkContext wi // at org.apache.spark.memory.TaskMemoryManager.getPage(TaskMemoryManager.java:384) // - java.lang.UnsupportedOperationException: Cannot grow BufferHolder by size -536870912 // because the size after growing exceeds size limitation 2147483632 - val e = intercept[SparkOutOfMemoryError] { - sorter.insertRecord(bytes, Platform.BYTE_ARRAY_OFFSET, 1, 0) - } - assert(e.getMessage == - "[UNABLE_TO_ACQUIRE_MEMORY] Unable to acquire 800 bytes of memory, got 400") - assert(e.getErrorClass == "UNABLE_TO_ACQUIRE_MEMORY") - assert(e.getSqlState == null) + checkError( + exception = intercept[SparkOutOfMemoryError] { + sorter.insertRecord(bytes, Platform.BYTE_ARRAY_OFFSET, 1, 0) + }, + errorClass = "UNABLE_TO_ACQUIRE_MEMORY", + parameters = Map("requestedBytes" -> "800", "receivedBytes" -> "400")) } } diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index ed8f085b8bf95..ad02f04bed90e 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -1825,10 +1825,10 @@ Install Apache YuniKorn: helm repo add yunikorn https://apache.github.io/yunikorn-release helm repo update kubectl create namespace yunikorn -helm install yunikorn yunikorn/yunikorn --namespace yunikorn --version 1.0.0 +helm install yunikorn yunikorn/yunikorn --namespace yunikorn --version 1.1.0 ``` -The above steps will install YuniKorn v1.0.0 on an existing Kubernetes cluster. +The above steps will install YuniKorn v1.1.0 on an existing Kubernetes cluster. ##### Get started diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index e77f65fb7d6d3..4a3a33b51f25f 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -89,7 +89,14 @@ object MimaExcludes { // [SPARK-40324][SQL] Provide query context in AnalysisException ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.AnalysisException.copy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.AnalysisException.withPosition") + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.AnalysisException.withPosition"), + + // [SPARK-40400][SQL] Pass error message parameters to exceptions as a map + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.AnalysisException.messageParameters"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.AnalysisException.copy$default$7"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.AnalysisException.copy"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.AnalysisException.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.SparkException.this") ) // Defulat exclude rules diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index bf6792313ae2d..9ad35a3db6868 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen; +import org.apache.spark.sql.errors.QueryExecutionErrors; import org.apache.spark.sql.types.Decimal; import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.ByteArrayMethods; @@ -55,10 +56,18 @@ public void initialize(int numElements) { this.startingOffset = cursor(); + long fixedPartInBytesLong = + ByteArrayMethods.roundNumberOfBytesToNearestWord((long) elementSize * numElements); + long totalInitialSize = headerInBytes + fixedPartInBytesLong; + + if (totalInitialSize > Integer.MAX_VALUE) { + throw QueryExecutionErrors.tooManyArrayElementsError(numElements, elementSize); + } + + // it's now safe to cast fixedPartInBytesLong and totalInitialSize to int + int fixedPartInBytes = (int) fixedPartInBytesLong; // Grows the global buffer ahead for header and fixed size data. - int fixedPartInBytes = - ByteArrayMethods.roundNumberOfBytesToNearestWord(elementSize * numElements); - holder.grow(headerInBytes + fixedPartInBytes); + holder.grow((int)totalInitialSize); // Write numElements and clear out null bits to header Platform.putLong(getBuffer(), startingOffset, numElements); 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 bb2ed251f3efd..4b27dfc00c2aa 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 @@ -37,28 +37,33 @@ class AnalysisException protected[sql] ( val cause: Option[Throwable] = None, val errorClass: Option[String] = None, val errorSubClass: Option[String] = None, - val messageParameters: Array[String] = Array.empty, + val messageParameters: Map[String, String] = Map.empty, val context: Array[QueryContext] = Array.empty) extends Exception(message, cause.orNull) with SparkThrowable with Serializable { // Needed for binary compatibility - protected[sql] def this(message: String, - line: Option[Int], - startPosition: Option[Int], - plan: Option[LogicalPlan], - cause: Option[Throwable], - errorClass: Option[String], - messageParameters: Array[String]) = - this(message = message, - line = line, - startPosition = startPosition, - plan = plan, - cause = cause, - errorClass, - errorSubClass = None, - messageParameters = messageParameters) - - def this(errorClass: String, messageParameters: Array[String], cause: Option[Throwable]) = + protected[sql] def this( + message: String, + line: Option[Int], + startPosition: Option[Int], + plan: Option[LogicalPlan], + cause: Option[Throwable], + errorClass: Option[String], + messageParameters: Map[String, String]) = + this( + message = message, + line = line, + startPosition = startPosition, + plan = plan, + cause = cause, + errorClass, + errorSubClass = None, + messageParameters = messageParameters) + + def this( + errorClass: String, + messageParameters: Map[String, String], + cause: Option[Throwable]) = this( SparkThrowableHelper.getMessage(errorClass, null, messageParameters), errorClass = Some(errorClass), @@ -68,7 +73,7 @@ class AnalysisException protected[sql] ( def this( errorClass: String, - messageParameters: Array[String], + messageParameters: Map[String, String], context: Array[QueryContext], summary: String) = this( @@ -79,12 +84,17 @@ class AnalysisException protected[sql] ( cause = null, context = context) - def this(errorClass: String, messageParameters: Array[String]) = - this(errorClass = errorClass, messageParameters = messageParameters, cause = None) + def this( + errorClass: String, + messageParameters: Map[String, String]) = + this( + errorClass = errorClass, + messageParameters = messageParameters, + cause = None) def this( errorClass: String, - messageParameters: Array[String], + messageParameters: Map[String, String], origin: Origin) = this( SparkThrowableHelper.getMessage(errorClass, null, messageParameters), @@ -98,7 +108,7 @@ class AnalysisException protected[sql] ( def this( errorClass: String, errorSubClass: String, - messageParameters: Array[String]) = + messageParameters: Map[String, String]) = this( SparkThrowableHelper.getMessage(errorClass, errorSubClass, messageParameters), errorClass = Some(errorClass), @@ -108,7 +118,7 @@ class AnalysisException protected[sql] ( def this( errorClass: String, errorSubClass: String, - messageParameters: Array[String], + messageParameters: Map[String, String], origin: Origin) = this( SparkThrowableHelper.getMessage(errorClass, errorSubClass, messageParameters), @@ -126,8 +136,8 @@ class AnalysisException protected[sql] ( plan: Option[LogicalPlan] = this.plan, cause: Option[Throwable] = this.cause, errorClass: Option[String] = this.errorClass, - messageParameters: Array[String] = this.messageParameters, - context: Array[QueryContext] = Array.empty): AnalysisException = + messageParameters: Map[String, String] = this.messageParameters, + context: Array[QueryContext] = this.context): AnalysisException = new AnalysisException(message, line, startPosition, plan, cause, errorClass, errorSubClass, messageParameters, context) @@ -155,7 +165,12 @@ class AnalysisException protected[sql] ( message } - override def getMessageParameters: Array[String] = messageParameters + override def getMessageParameters: Array[String] = { + errorClass.map { ec => + SparkThrowableHelper.getMessageParameters(ec, errorSubClass.orNull, messageParameters) + }.getOrElse(Array.empty) + } + override def getErrorClass: String = errorClass.orNull override def getErrorSubClass: String = errorSubClass.orNull override def getQueryContext: Array[QueryContext] = context 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 b4d927adecb67..6fc9d756c998d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -209,8 +209,7 @@ class Analyzer(override val catalogManager: CatalogManager) analyzed } catch { case e: AnalysisException => - val ae = e.copy(plan = Option(analyzed), - context = analyzed.origin.getQueryContext) + val ae = e.copy(plan = Option(analyzed)) ae.setStackTrace(e.getStackTrace) throw ae } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCheckResult.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCheckResult.scala index b6448b2f73484..2041eaa9ad56b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCheckResult.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCheckResult.scala @@ -51,7 +51,7 @@ object TypeCheckResult { */ case class DataTypeMismatch( errorSubClass: String, - messageParameters: Array[String]) + messageParameters: Map[String, String]) extends TypeCheckResult { def isSuccess: Boolean = false } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala index be6dcfac269a5..4e32e506c6881 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala @@ -50,7 +50,7 @@ package object analysis { throw new AnalysisException(msg, t.origin.line, t.origin.startPosition, cause = Some(cause)) } - def failAnalysis(errorClass: String, messageParameters: Array[String]): Nothing = { + def failAnalysis(errorClass: String, messageParameters: Map[String, String]): Nothing = { throw new AnalysisException( errorClass = errorClass, messageParameters = messageParameters, @@ -61,7 +61,7 @@ package object analysis { throw new AnalysisException( errorClass = "DATATYPE_MISMATCH", errorSubClass = mismatch.errorSubClass, - messageParameters = toSQLExpr(expr) +: mismatch.messageParameters, + messageParameters = mismatch.messageParameters + ("sqlExpr" -> toSQLExpr(expr)), origin = t.origin) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index e5c3f7e80e0f3..96ea758fb1bfd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -652,7 +652,7 @@ case object UnresolvedSeed extends LeafExpression with Unevaluable { */ case class TempResolvedColumn(child: Expression, nameParts: Seq[String]) extends UnaryExpression with Unevaluable { - override lazy val preCanonicalized = child.preCanonicalized + override lazy val canonicalized = child.canonicalized override def dataType: DataType = child.dataType override protected def withNewChildInternal(newChild: Expression): Expression = copy(child = newChild) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InvalidUDFClassException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InvalidUDFClassException.scala index 4d83663fd31fb..6b06548e933cb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InvalidUDFClassException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InvalidUDFClassException.scala @@ -27,11 +27,11 @@ import org.apache.spark.sql.AnalysisException class InvalidUDFClassException private[sql]( message: String, errorClass: Option[String] = None, - messageParameters: Array[String] = Array.empty) + messageParameters: Map[String, String] = Map.empty) extends AnalysisException( message = message, errorClass = errorClass, messageParameters = messageParameters) { - def this(errorClass: String, messageParameters: Array[String]) = + def this(errorClass: String, messageParameters: Map[String, String]) = this( SparkThrowableHelper.getMessage(errorClass, null, messageParameters), Some(errorClass), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala deleted file mode 100644 index 329ed3f20b727..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala +++ /dev/null @@ -1,70 +0,0 @@ -/* - * 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.expressions - -/** - * Reorders adjacent commutative operators such as [[And]] in the expression tree, according to - * the `hashCode` of non-commutative nodes, to remove cosmetic variations. Caller side should only - * call it on the root node of an expression tree that needs to be canonicalized. - */ -object Canonicalize { - /** Collects adjacent commutative operations. */ - private def gatherCommutative( - e: Expression, - f: PartialFunction[Expression, Seq[Expression]]): Seq[Expression] = e match { - case c if f.isDefinedAt(c) => f(c).flatMap(gatherCommutative(_, f)) - case other => reorderCommutativeOperators(other) :: Nil - } - - /** Orders a set of commutative operations by their hash code. */ - private def orderCommutative( - e: Expression, - f: PartialFunction[Expression, Seq[Expression]]): Seq[Expression] = - gatherCommutative(e, f).sortBy(_.hashCode()) - - def reorderCommutativeOperators(e: Expression): Expression = e match { - // TODO: do not reorder consecutive `Add`s or `Multiply`s with different `failOnError` flags - case a @ Add(_, _, f) => - orderCommutative(a, { case Add(l, r, _) => Seq(l, r) }).reduce(Add(_, _, f)) - case m @ Multiply(_, _, f) => - orderCommutative(m, { case Multiply(l, r, _) => Seq(l, r) }).reduce(Multiply(_, _, f)) - - case o @ Or(l, r) if l.deterministic && r.deterministic => - orderCommutative(o, { case Or(l, r) if l.deterministic && r.deterministic => Seq(l, r) }) - .reduce(Or) - case a @ And(l, r) if l.deterministic && r.deterministic => - orderCommutative(a, { case And(l, r) if l.deterministic && r.deterministic => Seq(l, r)}) - .reduce(And) - - case o: BitwiseOr => - orderCommutative(o, { case BitwiseOr(l, r) => Seq(l, r) }).reduce(BitwiseOr) - case a: BitwiseAnd => - orderCommutative(a, { case BitwiseAnd(l, r) => Seq(l, r) }).reduce(BitwiseAnd) - case x: BitwiseXor => - orderCommutative(x, { case BitwiseXor(l, r) => Seq(l, r) }).reduce(BitwiseXor) - - case g: Greatest => - val newChildren = orderCommutative(g, { case Greatest(children) => children }) - Greatest(newChildren) - case l: Least => - val newChildren = orderCommutative(l, { case Least(children) => children }) - Least(newChildren) - - case _ => e.withNewChildren(e.children.map(reorderCommutativeOperators)) - } -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 356d5ed62cd9e..d8642d22af002 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -563,8 +563,8 @@ case class Cast( override lazy val resolved: Boolean = childrenResolved && checkInputDataTypes().isSuccess && (!needsTimeZone || timeZoneId.isDefined) - override lazy val preCanonicalized: Expression = { - val basic = withNewChildren(Seq(child.preCanonicalized)).asInstanceOf[Cast] + override lazy val canonicalized: Expression = { + val basic = withNewChildren(Seq(child.canonicalized)).asInstanceOf[Cast] if (timeZoneId.isDefined && !needsTimeZone) { basic.withTimeZone(null) } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala index b268818568056..dd9e9307e74e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala @@ -74,11 +74,11 @@ case class DynamicPruningSubquery( override def toString: String = s"dynamicpruning#${exprId.id} $conditionString" - override lazy val preCanonicalized: DynamicPruning = { + override lazy val canonicalized: DynamicPruning = { copy( - pruningKey = pruningKey.preCanonicalized, + pruningKey = pruningKey.canonicalized, buildQuery = buildQuery.canonicalized, - buildKeys = buildKeys.map(_.preCanonicalized), + buildKeys = buildKeys.map(_.canonicalized), exprId = ExprId(0)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index a85e7d2efdb3b..6df03aa8e84e9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -224,43 +224,28 @@ abstract class Expression extends TreeNode[Expression] { */ def childrenResolved: Boolean = children.forall(_.resolved) - // Expression canonicalization is done in 2 phases: - // 1. Recursively canonicalize each node in the expression tree. This does not change the tree - // structure and is more like "node-local" canonicalization. - // 2. Find adjacent commutative operators in the expression tree, reorder them to get a - // static order and remove cosmetic variations. This may change the tree structure - // dramatically and is more like a "global" canonicalization. - // - // The first phase is done by `preCanonicalized`. It's a `lazy val` which recursively calls - // `preCanonicalized` on the children. This means that almost every node in the expression tree - // will instantiate the `preCanonicalized` variable, which is good for performance as you can - // reuse the canonicalization result of the children when you construct a new expression node. - // - // The second phase is done by `canonicalized`, which simply calls `Canonicalize` and is kind of - // the actual "user-facing API" of expression canonicalization. Only the root node of the - // expression tree will instantiate the `canonicalized` variable. This is different from - // `preCanonicalized`, because `canonicalized` does "global" canonicalization and most of the time - // you cannot reuse the canonicalization result of the children. - - /** - * An internal lazy val to implement expression canonicalization. It should only be called in - * `canonicalized`, or in subclass's `preCanonicalized` when the subclass overrides this lazy val - * to provide custom canonicalization logic. - */ - lazy val preCanonicalized: Expression = { - val canonicalizedChildren = children.map(_.preCanonicalized) - withNewChildren(canonicalizedChildren) - } - /** * Returns an expression where a best effort attempt has been made to transform `this` in a way * that preserves the result but removes cosmetic variations (case sensitivity, ordering for - * commutative operations, etc.) See [[Canonicalize]] for more details. + * commutative operations, etc.). * * `deterministic` expressions where `this.canonicalized == other.canonicalized` will always * evaluate to the same result. + * + * The process of canonicalization is a one pass, bottum-up expression tree computation based on + * canonicalizing children before canonicalizing the current node. There is one exception though, + * as adjacent, same class [[CommutativeExpression]]s canonicalazion happens in a way that calling + * `canonicalized` on the root: + * 1. Gathers and canonicalizes the non-commutative (or commutative but not same class) child + * expressions of the adjacent expressions. + * 2. Reorder the canonicalized child expressions by their hashcode. + * This means that the lazy `cannonicalized` is called and computed only on the root of the + * adjacent expressions. */ - lazy val canonicalized: Expression = Canonicalize.reorderCommutativeOperators(preCanonicalized) + lazy val canonicalized: Expression = { + val canonicalizedChildren = children.map(_.canonicalized) + withNewChildren(canonicalizedChildren) + } /** * Returns true when two expressions will always compute the same result, even if they differ @@ -364,7 +349,7 @@ trait RuntimeReplaceable extends Expression { // As this expression gets replaced at optimization with its `child" expression, // two `RuntimeReplaceable` are considered to be semantically equal if their "child" expressions // are semantically equal. - override lazy val preCanonicalized: Expression = replacement.preCanonicalized + override lazy val canonicalized: Expression = replacement.canonicalized final override def eval(input: InternalRow = null): Any = throw QueryExecutionErrors.cannotEvaluateExpressionError(this) @@ -763,11 +748,15 @@ abstract class BinaryOperator extends BinaryExpression with ExpectsInputTypes wi if (!left.dataType.sameType(right.dataType)) { DataTypeMismatch( errorSubClass = "BINARY_OP_DIFF_TYPES", - messageParameters = Array(toSQLType(left.dataType), toSQLType(right.dataType))) + messageParameters = Map( + "left" -> toSQLType(left.dataType), + "right" -> toSQLType(right.dataType))) } else if (!inputType.acceptsType(left.dataType)) { DataTypeMismatch( errorSubClass = "BINARY_OP_WRONG_TYPE", - messageParameters = Array(toSQLType(inputType), toSQLType(left.dataType))) + messageParameters = Map( + "inputType" -> toSQLType(inputType), + "actualDataType" -> toSQLType(left.dataType))) } else { TypeCheckResult.TypeCheckSuccess } @@ -1176,3 +1165,21 @@ trait ComplexTypeMergingExpression extends Expression { trait UserDefinedExpression { def name: String } + +trait CommutativeExpression extends Expression { + /** Collects adjacent commutative operations. */ + private def gatherCommutative( + e: Expression, + f: PartialFunction[CommutativeExpression, Seq[Expression]]): Seq[Expression] = e match { + case c: CommutativeExpression if f.isDefinedAt(c) => f(c).flatMap(gatherCommutative(_, f)) + case other => other.canonicalized :: Nil + } + + /** + * Reorders adjacent commutative operators such as [[And]] in the expression tree, according to + * the `hashCode` of non-commutative nodes, to remove cosmetic variations. + */ + protected def orderCommutative( + f: PartialFunction[CommutativeExpression, Seq[Expression]]): Seq[Expression] = + gatherCommutative(this, f).sortBy(_.hashCode()) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PythonUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PythonUDF.scala index b84810c3f4bb4..6b9017a01db36 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PythonUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PythonUDF.scala @@ -71,8 +71,8 @@ case class PythonUDF( override def nullable: Boolean = true - override lazy val preCanonicalized: Expression = { - val canonicalizedChildren = children.map(_.preCanonicalized) + override lazy val canonicalized: Expression = { + val canonicalizedChildren = children.map(_.canonicalized) // `resultId` can be seen as cosmetic variation in PythonUDF, as it doesn't affect the result. this.copy(resultId = ExprId(-1)).withNewChildren(canonicalizedChildren) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 832b2656f0c96..f8ff5f583f602 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -64,10 +64,10 @@ case class ScalaUDF( override def name: String = udfName.getOrElse("UDF") - override lazy val preCanonicalized: Expression = { + override lazy val canonicalized: Expression = { // SPARK-32307: `ExpressionEncoder` can't be canonicalized, and technically we don't // need it to identify a `ScalaUDF`. - copy(children = children.map(_.preCanonicalized), inputEncoders = Nil, outputEncoder = None) + copy(children = children.map(_.canonicalized), inputEncoders = Nil, outputEncoder = None) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala index a9168e58dbdb3..92bc930902a07 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala @@ -126,7 +126,7 @@ case class AggregateExpression( def filterAttributes: AttributeSet = filter.map(_.references).getOrElse(AttributeSet.empty) // We compute the same thing regardless of our final result. - override lazy val preCanonicalized: Expression = { + override lazy val canonicalized: Expression = { val normalizedAggFunc = mode match { // For PartialMerge or Final mode, the input to the `aggregateFunction` is aggregate buffers, // and the actual children of `aggregateFunction` is not used, here we normalize the expr id. @@ -137,10 +137,10 @@ case class AggregateExpression( } AggregateExpression( - normalizedAggFunc.preCanonicalized.asInstanceOf[AggregateFunction], + normalizedAggFunc.canonicalized.asInstanceOf[AggregateFunction], mode, isDistinct, - filter.map(_.preCanonicalized), + filter.map(_.canonicalized), ExprId(0)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 41973e95569c8..72622da7eecdf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -417,7 +417,8 @@ object BinaryArithmetic { case class Add( left: Expression, right: Expression, - evalMode: EvalMode.Value = EvalMode.fromSQLConf(SQLConf.get)) extends BinaryArithmetic { + evalMode: EvalMode.Value = EvalMode.fromSQLConf(SQLConf.get)) extends BinaryArithmetic + with CommutativeExpression { def this(left: Expression, right: Expression) = this(left, right, EvalMode.fromSQLConf(SQLConf.get)) @@ -473,6 +474,11 @@ case class Add( override protected def withNewChildrenInternal(newLeft: Expression, newRight: Expression): Add = copy(left = newLeft, right = newRight) + + override lazy val canonicalized: Expression = { + // TODO: do not reorder consecutive `Add`s with different `evalMode` + orderCommutative({ case Add(l, r, _) => Seq(l, r) }).reduce(Add(_, _, evalMode)) + } } @ExpressionDescription( @@ -563,7 +569,8 @@ case class Subtract( case class Multiply( left: Expression, right: Expression, - evalMode: EvalMode.Value = EvalMode.fromSQLConf(SQLConf.get)) extends BinaryArithmetic { + evalMode: EvalMode.Value = EvalMode.fromSQLConf(SQLConf.get)) extends BinaryArithmetic + with CommutativeExpression { def this(left: Expression, right: Expression) = this(left, right, EvalMode.fromSQLConf(SQLConf.get)) @@ -612,6 +619,11 @@ case class Multiply( override protected def withNewChildrenInternal( newLeft: Expression, newRight: Expression): Multiply = copy(left = newLeft, right = newRight) + + override lazy val canonicalized: Expression = { + // TODO: do not reorder consecutive `Multiply`s with different `evalMode` + orderCommutative({ case Multiply(l, r, _) => Seq(l, r) }).reduce(Multiply(_, _, evalMode)) + } } // Common base trait for Divide and Remainder, since these two classes are almost identical @@ -1176,7 +1188,8 @@ case class Pmod( """, since = "1.5.0", group = "math_funcs") -case class Least(children: Seq[Expression]) extends ComplexTypeMergingExpression { +case class Least(children: Seq[Expression]) extends ComplexTypeMergingExpression + with CommutativeExpression { override def nullable: Boolean = children.forall(_.nullable) override def foldable: Boolean = children.forall(_.foldable) @@ -1239,6 +1252,10 @@ case class Least(children: Seq[Expression]) extends ComplexTypeMergingExpression override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Least = copy(children = newChildren) + + override lazy val canonicalized: Expression = { + Least(orderCommutative({ case Least(children) => children })) + } } /** @@ -1254,7 +1271,8 @@ case class Least(children: Seq[Expression]) extends ComplexTypeMergingExpression """, since = "1.5.0", group = "math_funcs") -case class Greatest(children: Seq[Expression]) extends ComplexTypeMergingExpression { +case class Greatest(children: Seq[Expression]) extends ComplexTypeMergingExpression + with CommutativeExpression { override def nullable: Boolean = children.forall(_.nullable) override def foldable: Boolean = children.forall(_.foldable) @@ -1317,4 +1335,8 @@ case class Greatest(children: Seq[Expression]) extends ComplexTypeMergingExpress override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Greatest = copy(children = newChildren) + + override lazy val canonicalized: Expression = { + Greatest(orderCommutative({ case Greatest(children) => children })) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala index a178500fba830..70c3d11dedabe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala @@ -36,7 +36,8 @@ import org.apache.spark.sql.types._ """, since = "1.4.0", group = "bitwise_funcs") -case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithmetic { +case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithmetic + with CommutativeExpression { protected override val evalMode: EvalMode.Value = EvalMode.LEGACY @@ -59,6 +60,10 @@ case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithme override protected def withNewChildrenInternal( newLeft: Expression, newRight: Expression): BitwiseAnd = copy(left = newLeft, right = newRight) + + override lazy val canonicalized: Expression = { + orderCommutative({ case BitwiseAnd(l, r) => Seq(l, r) }).reduce(BitwiseAnd) + } } /** @@ -75,7 +80,8 @@ case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithme """, since = "1.4.0", group = "bitwise_funcs") -case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmetic { +case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmetic + with CommutativeExpression { protected override val evalMode: EvalMode.Value = EvalMode.LEGACY @@ -98,6 +104,10 @@ case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmet override protected def withNewChildrenInternal( newLeft: Expression, newRight: Expression): BitwiseOr = copy(left = newLeft, right = newRight) + + override lazy val canonicalized: Expression = { + orderCommutative({ case BitwiseOr(l, r) => Seq(l, r) }).reduce(BitwiseOr) + } } /** @@ -114,7 +124,8 @@ case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmet """, since = "1.4.0", group = "bitwise_funcs") -case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithmetic { +case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithmetic + with CommutativeExpression { protected override val evalMode: EvalMode.Value = EvalMode.LEGACY @@ -137,6 +148,10 @@ case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithme override protected def withNewChildrenInternal( newLeft: Expression, newRight: Expression): BitwiseXor = copy(left = newLeft, right = newRight) + + override lazy val canonicalized: Expression = { + orderCommutative({ case BitwiseXor(l, r) => Seq(l, r) }).reduce(BitwiseXor) + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala index 923e4ee8ebe3d..41c410cc45a54 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala @@ -107,8 +107,8 @@ case class GetStructField(child: Expression, ordinal: Int, name: Option[String] lazy val childSchema = child.dataType.asInstanceOf[StructType] - override lazy val preCanonicalized: Expression = { - copy(child = child.preCanonicalized, name = None) + override lazy val canonicalized: Expression = { + copy(child = child.canonicalized, name = None) } override def dataType: DataType = childSchema(ordinal).dataType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala index f5379336bca2f..5b8b4b3f621ee 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala @@ -208,7 +208,7 @@ trait HigherOrderFunction extends Expression with ExpectsInputTypes { } } - override lazy val preCanonicalized: Expression = { + override lazy val canonicalized: Expression = { var currExprId = -1 val argumentMap = functions.flatMap(_.collect { case l: NamedLambdaVariable => @@ -221,7 +221,7 @@ trait HigherOrderFunction extends Expression with ExpectsInputTypes { val newExprId = argumentMap(l.exprId) NamedLambdaVariable("none", l.dataType, l.nullable, exprId = ExprId(newExprId), null) } - val canonicalizedChildren = cleaned.children.map(_.preCanonicalized) + val canonicalizedChildren = cleaned.children.map(_.canonicalized) withNewChildren(canonicalizedChildren) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index cc16e1fa5866b..4181edcb8c601 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -295,7 +295,7 @@ case class AttributeReference( h } - override lazy val preCanonicalized: Expression = { + override lazy val canonicalized: Expression = { AttributeReference("none", dataType)(exprId) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index cd062e3e46e3c..21f65cb3402ee 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -328,8 +328,8 @@ case class Not(child: Expression) final override val nodePatterns: Seq[TreePattern] = Seq(NOT) - override lazy val preCanonicalized: Expression = { - withNewChildren(Seq(child.preCanonicalized)) match { + override lazy val canonicalized: Expression = { + withNewChildren(Seq(child.canonicalized)) match { case Not(GreaterThan(l, r)) => LessThanOrEqual(l, r) case Not(LessThan(l, r)) => GreaterThanOrEqual(l, r) case Not(GreaterThanOrEqual(l, r)) => LessThan(l, r) @@ -466,8 +466,8 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate { final override val nodePatterns: Seq[TreePattern] = Seq(IN) - override lazy val preCanonicalized: Expression = { - val basic = withNewChildren(children.map(_.preCanonicalized)).asInstanceOf[In] + override lazy val canonicalized: Expression = { + val basic = withNewChildren(children.map(_.canonicalized)).asInstanceOf[In] if (list.size > 1) { basic.copy(list = basic.list.sortBy(_.hashCode())) } else { @@ -736,7 +736,8 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with """, since = "1.0.0", group = "predicate_funcs") -case class And(left: Expression, right: Expression) extends BinaryOperator with Predicate { +case class And(left: Expression, right: Expression) extends BinaryOperator with Predicate + with CommutativeExpression { override def inputType: AbstractDataType = BooleanType @@ -807,6 +808,10 @@ case class And(left: Expression, right: Expression) extends BinaryOperator with override protected def withNewChildrenInternal(newLeft: Expression, newRight: Expression): And = copy(left = newLeft, right = newRight) + + override lazy val canonicalized: Expression = { + orderCommutative({ case And(l, r) => Seq(l, r) }).reduce(And) + } } @ExpressionDescription( @@ -824,7 +829,8 @@ case class And(left: Expression, right: Expression) extends BinaryOperator with """, since = "1.0.0", group = "predicate_funcs") -case class Or(left: Expression, right: Expression) extends BinaryOperator with Predicate { +case class Or(left: Expression, right: Expression) extends BinaryOperator with Predicate + with CommutativeExpression { override def inputType: AbstractDataType = BooleanType @@ -896,6 +902,10 @@ case class Or(left: Expression, right: Expression) extends BinaryOperator with P override protected def withNewChildrenInternal(newLeft: Expression, newRight: Expression): Or = copy(left = newLeft, right = newRight) + + override lazy val canonicalized: Expression = { + orderCommutative({ case Or(l, r) => Seq(l, r) }).reduce(Or) + } } @@ -907,8 +917,8 @@ abstract class BinaryComparison extends BinaryOperator with Predicate { final override val nodePatterns: Seq[TreePattern] = Seq(BINARY_COMPARISON) - override lazy val preCanonicalized: Expression = { - withNewChildren(children.map(_.preCanonicalized)) match { + override lazy val canonicalized: Expression = { + withNewChildren(children.map(_.canonicalized)) match { case EqualTo(l, r) if l.hashCode() > r.hashCode() => EqualTo(r, l) case EqualNullSafe(l, r) if l.hashCode() > r.hashCode() => EqualNullSafe(r, l) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index 71b36fa8ef9ba..0e091cdbe0239 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -263,12 +263,12 @@ case class ScalarSubquery( override def nullable: Boolean = true override def withNewPlan(plan: LogicalPlan): ScalarSubquery = copy(plan = plan) override def toString: String = s"scalar-subquery#${exprId.id} $conditionString" - override lazy val preCanonicalized: Expression = { + override lazy val canonicalized: Expression = { ScalarSubquery( plan.canonicalized, - outerAttrs.map(_.preCanonicalized), + outerAttrs.map(_.canonicalized), ExprId(0), - joinCond.map(_.preCanonicalized)) + joinCond.map(_.canonicalized)) } override protected def withNewChildrenInternal( @@ -305,12 +305,12 @@ case class LateralSubquery( override def nullable: Boolean = true override def withNewPlan(plan: LogicalPlan): LateralSubquery = copy(plan = plan) override def toString: String = s"lateral-subquery#${exprId.id} $conditionString" - override lazy val preCanonicalized: Expression = { + override lazy val canonicalized: Expression = { LateralSubquery( plan.canonicalized, - outerAttrs.map(_.preCanonicalized), + outerAttrs.map(_.canonicalized), ExprId(0), - joinCond.map(_.preCanonicalized)) + joinCond.map(_.canonicalized)) } override protected def withNewChildrenInternal( @@ -350,13 +350,13 @@ case class ListQuery( override def nullable: Boolean = false override def withNewPlan(plan: LogicalPlan): ListQuery = copy(plan = plan) override def toString: String = s"list#${exprId.id} $conditionString" - override lazy val preCanonicalized: Expression = { + override lazy val canonicalized: Expression = { ListQuery( plan.canonicalized, - outerAttrs.map(_.preCanonicalized), + outerAttrs.map(_.canonicalized), ExprId(0), - childOutputs.map(_.preCanonicalized.asInstanceOf[Attribute]), - joinCond.map(_.preCanonicalized)) + childOutputs.map(_.canonicalized.asInstanceOf[Attribute]), + joinCond.map(_.canonicalized)) } override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): ListQuery = @@ -402,12 +402,12 @@ case class Exists( override def nullable: Boolean = false override def withNewPlan(plan: LogicalPlan): Exists = copy(plan = plan) override def toString: String = s"exists#${exprId.id} $conditionString" - override lazy val preCanonicalized: Expression = { + override lazy val canonicalized: Expression = { Exists( plan.canonicalized, - outerAttrs.map(_.preCanonicalized), + outerAttrs.map(_.canonicalized), ExprId(0), - joinCond.map(_.preCanonicalized)) + joinCond.map(_.canonicalized)) } override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Exists = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala index 9e864d036efd3..fd7a87087ddd2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.trees.TreeNodeTag -import org.apache.spark.sql.catalyst.trees.TreePattern.{LOCAL_RELATION, TRUE_OR_FALSE_LITERAL} +import org.apache.spark.sql.catalyst.trees.TreePattern.{LOCAL_RELATION, REPARTITION_OPERATION, TRUE_OR_FALSE_LITERAL} /** * The base class of two rules in the normal and AQE Optimizer. It simplifies query plans with @@ -185,13 +185,20 @@ abstract class PropagateEmptyRelationBase extends Rule[LogicalPlan] with CastSup * Add a [[ROOT_REPARTITION]] tag for the root user-specified repartition so this rule can * skip optimize it. */ - private def addTagForRootRepartition(plan: LogicalPlan): LogicalPlan = plan match { - case p: Project => p.mapChildren(addTagForRootRepartition) - case f: Filter => f.mapChildren(addTagForRootRepartition) - case r if userSpecifiedRepartition(r) => - r.setTagValue(ROOT_REPARTITION, ()) - r - case _ => plan + private def addTagForRootRepartition(plan: LogicalPlan): LogicalPlan = { + if (!plan.containsPattern(REPARTITION_OPERATION)) { + return plan + } + + plan match { + case p: Project => p.mapChildren(addTagForRootRepartition) + case f: Filter => f.mapChildren(addTagForRootRepartition) + case d: DeserializeToObject => d.mapChildren(addTagForRootRepartition) + case r if userSpecifiedRepartition(r) => + r.setTagValue(ROOT_REPARTITION, ()) + r + case _ => plan + } } override def apply(plan: LogicalPlan): LogicalPlan = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index 5eaef76a74711..4b2ee09804c52 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -238,7 +238,7 @@ class ParseException( val stop: Origin, errorClass: Option[String] = None, errorSubClass: Option[String] = None, - messageParameters: Array[String] = Array.empty, + messageParameters: Map[String, String] = Map.empty, queryContext: Array[QueryContext] = ParseException.getQueryContext()) extends AnalysisException( message, @@ -257,7 +257,7 @@ class ParseException( ParserUtils.position(ctx.getStop)) } - def this(errorClass: String, messageParameters: Array[String], ctx: ParserRuleContext) = + def this(errorClass: String, messageParameters: Map[String, String], ctx: ParserRuleContext) = this(Option(ParserUtils.command(ctx)), SparkThrowableHelper.getMessage(errorClass, null, messageParameters), ParserUtils.position(ctx.getStart), @@ -266,11 +266,13 @@ class ParseException( None, messageParameters) - def this(errorClass: String, - errorSubClass: String, - messageParameters: Array[String], - ctx: ParserRuleContext) = - this(Option(ParserUtils.command(ctx)), + def this( + errorClass: String, + errorSubClass: String, + messageParameters: Map[String, String], + ctx: ParserRuleContext) = + this( + Option(ParserUtils.command(ctx)), SparkThrowableHelper.getMessage(errorClass, errorSubClass, messageParameters), ParserUtils.position(ctx.getStart), ParserUtils.position(ctx.getStop), @@ -284,7 +286,7 @@ class ParseException( start: Origin, stop: Origin, errorClass: String, - messageParameters: Array[String]) = + messageParameters: Map[String, String]) = this( command, SparkThrowableHelper.getMessage(errorClass, null, messageParameters), @@ -319,7 +321,7 @@ class ParseException( val (cls, subCls, params) = if (errorClass == Some("PARSE_SYNTAX_ERROR") && cmd.trim().isEmpty) { // PARSE_EMPTY_STATEMENT error class overrides the PARSE_SYNTAX_ERROR when cmd is empty - (Some("PARSE_EMPTY_STATEMENT"), None, Array[String]()) + (Some("PARSE_EMPTY_STATEMENT"), None, Map.empty[String, String]) } else { (errorClass, errorSubClass, messageParameters) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SparkParserErrorStrategy.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SparkParserErrorStrategy.scala index 1b0b68620737f..9cc8fa8dcf850 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SparkParserErrorStrategy.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SparkParserErrorStrategy.scala @@ -30,14 +30,14 @@ class SparkRecognitionException( input: IntStream, ctx: ParserRuleContext, val errorClass: Option[String] = None, - val messageParameters: Array[String] = Array.empty) + val messageParameters: Map[String, String] = Map.empty) extends RecognitionException(message, recognizer, input, ctx) { /** Construct from a given [[RecognitionException]], with additional error information. */ def this( recognitionException: RecognitionException, errorClass: String, - messageParameters: Array[String]) = + messageParameters: Map[String, String]) = this( recognitionException.getMessage, recognitionException.getRecognizer, @@ -50,7 +50,7 @@ class SparkRecognitionException( messageParameters) /** Construct with pure errorClass and messageParameter information. */ - def this(errorClass: String, messageParameters: Array[String]) = + def this(errorClass: String, messageParameters: Map[String, String]) = this("", null, null, null, Some(errorClass), messageParameters) } @@ -75,7 +75,9 @@ class SparkParserErrorStrategy() extends DefaultErrorStrategy { val exceptionWithErrorClass = new SparkRecognitionException( e, "PARSE_SYNTAX_ERROR", - Array(getTokenErrorDisplay(e.getOffendingToken), "")) + messageParameters = Map( + "error" -> getTokenErrorDisplay(e.getOffendingToken), + "hint" -> "")) recognizer.notifyErrorListeners(e.getOffendingToken, "", exceptionWithErrorClass) } @@ -83,7 +85,7 @@ class SparkParserErrorStrategy() extends DefaultErrorStrategy { val exceptionWithErrorClass = new SparkRecognitionException( e, "PARSE_SYNTAX_ERROR", - Array(getTokenErrorDisplay(e.getOffendingToken), "")) + Map("error" -> getTokenErrorDisplay(e.getOffendingToken), "hint" -> "")) recognizer.notifyErrorListeners(e.getOffendingToken, "", exceptionWithErrorClass) } @@ -94,7 +96,7 @@ class SparkParserErrorStrategy() extends DefaultErrorStrategy { val hint = ": extra input " + errorTokenDisplay val exceptionWithErrorClass = new SparkRecognitionException( "PARSE_SYNTAX_ERROR", - Array(errorTokenDisplay, hint)) + Map("error" -> errorTokenDisplay, "hint" -> hint)) recognizer.notifyErrorListeners(recognizer.getCurrentToken, "", exceptionWithErrorClass) } } @@ -105,7 +107,7 @@ class SparkParserErrorStrategy() extends DefaultErrorStrategy { val hint = ": missing " + getExpectedTokens(recognizer).toString(recognizer.getVocabulary) val exceptionWithErrorClass = new SparkRecognitionException( "PARSE_SYNTAX_ERROR", - Array(getTokenErrorDisplay(recognizer.getCurrentToken), hint)) + Map("error" -> getTokenErrorDisplay(recognizer.getCurrentToken), "hint" -> hint)) recognizer.notifyErrorListeners(recognizer.getCurrentToken, "", exceptionWithErrorClass) } } 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 b62ae2a5900c9..c1d8f0a4a8a51 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 @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, AttributeSet, CreateMap, Expression, GroupingID, NamedExpression, SpecifiedWindowFrame, WindowFrame, WindowFunction, WindowSpecDefinition} import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoStatement, Join, LogicalPlan, SerdeInfo, Window} -import org.apache.spark.sql.catalyst.trees.{Origin, SQLQueryContext, TreeNode} +import org.apache.spark.sql.catalyst.trees.{Origin, TreeNode} import org.apache.spark.sql.catalyst.util.{FailFastMode, ParseMode, PermissiveMode} import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ @@ -51,51 +51,59 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def groupingIDMismatchError(groupingID: GroupingID, groupByExprs: Seq[Expression]): Throwable = { new AnalysisException( errorClass = "GROUPING_ID_COLUMN_MISMATCH", - messageParameters = Array(groupingID.groupByExprs.mkString(","), groupByExprs.mkString(","))) + messageParameters = Map( + "groupingIdColumn" -> groupingID.groupByExprs.mkString(","), + "groupByColumns" -> groupByExprs.mkString(","))) } def groupingColInvalidError(groupingCol: Expression, groupByExprs: Seq[Expression]): Throwable = { new AnalysisException( errorClass = "GROUPING_COLUMN_MISMATCH", - messageParameters = Array(groupingCol.toString, groupByExprs.mkString(","))) + messageParameters = Map( + "grouping" -> groupingCol.toString, + "groupingColumns" -> groupByExprs.mkString(","))) } def groupingSizeTooLargeError(sizeLimit: Int): Throwable = { new AnalysisException( errorClass = "GROUPING_SIZE_LIMIT_EXCEEDED", - messageParameters = Array(sizeLimit.toString)) + messageParameters = Map("maxSize" -> sizeLimit.toString)) } def zeroArgumentIndexError(): Throwable = { new AnalysisException( errorClass = "INVALID_PARAMETER_VALUE", - messageParameters = Array( - "strfmt", toSQLId("format_string"), "expects %1$, %2$ and so on, but got %0$.")) + messageParameters = Map( + "parameter" -> "strfmt", + "functionName" -> toSQLId("format_string"), + "expected" -> "expects %1$, %2$ and so on, but got %0$.")) } def unorderablePivotColError(pivotCol: Expression): Throwable = { new AnalysisException( errorClass = "INCOMPARABLE_PIVOT_COLUMN", - messageParameters = Array(toSQLId(pivotCol.sql))) + messageParameters = Map("columnName" -> toSQLId(pivotCol.sql))) } def nonLiteralPivotValError(pivotVal: Expression): Throwable = { new AnalysisException( errorClass = "NON_LITERAL_PIVOT_VALUES", - messageParameters = Array(toSQLExpr(pivotVal))) + messageParameters = Map("expression" -> toSQLExpr(pivotVal))) } def pivotValDataTypeMismatchError(pivotVal: Expression, pivotCol: Expression): Throwable = { new AnalysisException( errorClass = "PIVOT_VALUE_DATA_TYPE_MISMATCH", - messageParameters = Array( - pivotVal.toString, pivotVal.dataType.simpleString, pivotCol.dataType.catalogString)) + messageParameters = Map( + "value" -> pivotVal.toString, + "valueType" -> pivotVal.dataType.simpleString, + "pivotType" -> pivotCol.dataType.catalogString)) } def unpivotRequiresValueColumns(): Throwable = { new AnalysisException( errorClass = "UNPIVOT_REQUIRES_VALUE_COLUMNS", - messageParameters = Array.empty) + messageParameters = Map.empty) } def unpivotValDataTypeMismatchError(values: Seq[NamedExpression]): Throwable = { @@ -108,45 +116,49 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { new AnalysisException( errorClass = "UNPIVOT_VALUE_DATA_TYPE_MISMATCH", - messageParameters = Array(dataTypes.mkString(", "))) + messageParameters = Map("types" -> dataTypes.mkString(", "))) } def unsupportedIfNotExistsError(tableName: String): Throwable = { new AnalysisException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "INSERT_PARTITION_SPEC_IF_NOT_EXISTS", - messageParameters = Array(toSQLId(tableName))) + messageParameters = Map("tableName" -> toSQLId(tableName))) } def nonPartitionColError(partitionName: String): Throwable = { new AnalysisException( errorClass = "NON_PARTITION_COLUMN", - messageParameters = Array(toSQLId(partitionName))) + messageParameters = Map("columnName" -> toSQLId(partitionName))) } def missingStaticPartitionColumn(staticName: String): Throwable = { new AnalysisException( errorClass = "MISSING_STATIC_PARTITION_COLUMN", - messageParameters = Array(staticName)) + messageParameters = Map("columnName" -> staticName)) } def nestedGeneratorError(trimmedNestedGenerator: Expression): Throwable = { new AnalysisException(errorClass = "UNSUPPORTED_GENERATOR", errorSubClass = "NESTED_IN_EXPRESSIONS", - messageParameters = Array(toSQLExpr(trimmedNestedGenerator))) + messageParameters = Map("expression" -> toSQLExpr(trimmedNestedGenerator))) } def moreThanOneGeneratorError(generators: Seq[Expression], clause: String): Throwable = { - new AnalysisException(errorClass = "UNSUPPORTED_GENERATOR", + new AnalysisException( + errorClass = "UNSUPPORTED_GENERATOR", errorSubClass = "MULTI_GENERATOR", - messageParameters = Array(clause, - generators.size.toString, generators.map(toSQLExpr).mkString(", "))) + messageParameters = Map( + "clause" -> clause, + "num" -> generators.size.toString, + "generators" -> generators.map(toSQLExpr).mkString(", "))) } def generatorOutsideSelectError(plan: LogicalPlan): Throwable = { - new AnalysisException(errorClass = "UNSUPPORTED_GENERATOR", + new AnalysisException( + errorClass = "UNSUPPORTED_GENERATOR", errorSubClass = "OUTSIDE_SELECT", - messageParameters = Array(plan.simpleString(SQLConf.get.maxToStringFields))) + messageParameters = Map("plan" -> plan.simpleString(SQLConf.get.maxToStringFields))) } def legacyStoreAssignmentPolicyError(): Throwable = { @@ -168,43 +180,49 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { colName: String, candidates: Seq[String], origin: Origin): Throwable = { + val commonParam = Map("objectName" -> toSQLId(colName)) + val proposalParam = if (candidates.isEmpty) { + Map.empty[String, String] + } else { + Map("proposal" -> candidates.take(5).map(toSQLId).mkString(", ")) + } new AnalysisException( errorClass = errorClass, errorSubClass = if (candidates.isEmpty) "WITHOUT_SUGGESTION" else "WITH_SUGGESTION", - messageParameters = Array.concat(Array(toSQLId(colName)), if (candidates.isEmpty) { - Array.empty - } else { - Array(candidates.take(5).map(toSQLId).mkString(", ")) - }), + messageParameters = commonParam ++ proposalParam, origin = origin ) } def unresolvedColumnError(columnName: String, proposal: Seq[String]): Throwable = { + val commonParam = Map("objectName" -> toSQLId(columnName)) + val proposalParam = if (proposal.isEmpty) { + Map.empty[String, String] + } else { + Map("proposal" -> proposal.take(5).map(toSQLId).mkString(", ")) + } new AnalysisException( errorClass = "UNRESOLVED_COLUMN", errorSubClass = if (proposal.isEmpty) "WITHOUT_SUGGESTION" else "WITH_SUGGESTION", - messageParameters = Array.concat(Array(toSQLId(columnName)), if (proposal.isEmpty) { - Array.empty - } else { - Array(proposal.take(5).map(toSQLId).mkString(", ")) - })) + messageParameters = commonParam ++ proposalParam) } def unresolvedFieldError( fieldName: String, columnPath: Seq[String], proposal: Seq[String]): Throwable = { + val commonParams = Map( + "fieldName" -> toSQLId(fieldName), + "columnPath" -> toSQLId(columnPath)) + val proposalParam = if (proposal.isEmpty) { + Map.empty[String, String] + } else { + Map("proposal" -> proposal.map(toSQLId).mkString(", ")) + } new AnalysisException( errorClass = "UNRESOLVED_FIELD", errorSubClass = if (proposal.isEmpty) "WITHOUT_SUGGESTION" else "WITH_SUGGESTION", - messageParameters = - Array.concat(Array(toSQLId(fieldName), toSQLId(columnPath)), if (proposal.isEmpty) { - Array.empty - } else { - Array(proposal.map(toSQLId).mkString(", ")) - }) - ) + messageParameters = commonParams ++ proposalParam) } def dataTypeMismatchForDeserializerError( @@ -212,8 +230,9 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { new AnalysisException( errorClass = "UNSUPPORTED_DESERIALIZER", errorSubClass = "DATA_TYPE_MISMATCH", - messageParameters = - Array(toSQLType(desiredType), toSQLType(dataType))) + messageParameters = Map( + "desiredType" -> toSQLType(desiredType), + "dataType" -> toSQLType(dataType))) } def fieldNumberMismatchForDeserializerError( @@ -221,22 +240,23 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { new AnalysisException( errorClass = "UNSUPPORTED_DESERIALIZER", errorSubClass = "FIELD_NUMBER_MISMATCH", - messageParameters = - Array(toSQLType(schema), (maxOrdinal + 1).toString)) + messageParameters = Map( + "schema" -> toSQLType(schema), + "ordinal" -> (maxOrdinal + 1).toString)) } def upCastFailureError( fromStr: String, from: Expression, to: DataType, walkedTypePath: Seq[String]): Throwable = { new AnalysisException( errorClass = "CANNOT_UP_CAST_DATATYPE", - messageParameters = Array( - fromStr, - toSQLType(from.dataType), - toSQLType(to), - s"The type path of the target object is:\n" + walkedTypePath.mkString("", "\n", "\n") + + messageParameters = Map( + "expression" -> fromStr, + "sourceType" -> toSQLType(from.dataType), + "targetType" -> toSQLType(to), + "details" -> (s"The type path of the target object is:\n" + + walkedTypePath.mkString("", "\n", "\n") + "You can either add an explicit cast to the input data or choose a higher precision " + - "type of the field in the target object" - ) + "type of the field in the target object")) ) } @@ -264,14 +284,14 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def groupingMustWithGroupingSetsOrCubeOrRollupError(): Throwable = { new AnalysisException( errorClass = "UNSUPPORTED_GROUPING_EXPRESSION", - messageParameters = Array.empty) + messageParameters = Map.empty) } def pandasUDFAggregateNotSupportedInPivotError(): Throwable = { new AnalysisException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "PANDAS_UDAF_IN_PIVOT", - messageParameters = Array[String]()) + messageParameters = Map.empty) } def aggregateExpressionRequiredForPivotError(sql: String): Throwable = { @@ -394,19 +414,25 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { expr: Expression): Throwable = { new AnalysisException( errorClass = "GROUP_BY_POS_REFERS_AGG_EXPR", - messageParameters = Array(index.toString, expr.sql)) + messageParameters = Map( + "index" -> index.toString, + "aggExpr" -> expr.sql)) } def groupByPositionRangeError(index: Int, size: Int): Throwable = { new AnalysisException( errorClass = "GROUP_BY_POS_OUT_OF_RANGE", - messageParameters = Array(index.toString, size.toString)) + messageParameters = Map( + "index" -> index.toString, + "size" -> size.toString)) } def generatorNotExpectedError(name: FunctionIdentifier, classCanonicalName: String): Throwable = { new AnalysisException(errorClass = "UNSUPPORTED_GENERATOR", errorSubClass = "NOT_GENERATOR", - messageParameters = Array(toSQLId(name.toString), classCanonicalName)) + messageParameters = Map( + "functionName" -> toSQLId(name.toString), + "classCanonicalName" -> classCanonicalName)) } def functionWithUnsupportedSyntaxError(prettyName: String, syntax: String): Throwable = { @@ -573,7 +599,9 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { new AnalysisException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "TABLE_OPERATION", - messageParameters = Array(toSQLId(nameParts), operation)) + messageParameters = Map( + "tableName" -> toSQLId(nameParts), + "operation" -> operation)) } def alterColumnWithV1TableCannotSpecifyNotNullError(): Throwable = { @@ -813,19 +841,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { s"The '$argName' parameter of function '$funcName' needs to be a $requiredType literal.") } - def invalidFormatInConversion( - argName: String, - funcName: String, - expected: String, - context: SQLQueryContext): Throwable = { - new AnalysisException( - errorClass = "INVALID_PARAMETER_VALUE", - messageParameters = - Array(toSQLId(argName), toSQLId(funcName), expected), - context = getQueryContext(context), - summary = getSummary(context)) - } - def invalidStringLiteralParameter( funcName: String, argName: String, @@ -863,7 +878,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def noHandlerForUDAFError(name: String): Throwable = { new InvalidUDFClassException( errorClass = "NO_HANDLER_FOR_UDAF", - messageParameters = Array(name)) + messageParameters = Map("functionName" -> name)) } def batchWriteCapabilityError( @@ -1440,7 +1455,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { groupAggPandasUDFNames: Seq[String]): Throwable = { new AnalysisException( errorClass = "INVALID_PANDAS_UDF_PLACEMENT", - messageParameters = Array(groupAggPandasUDFNames.map(toSQLId).mkString(", "))) + messageParameters = Map( + "functionList" -> groupAggPandasUDFNames.map(toSQLId).mkString(", "))) } def ambiguousAttributesInSelfJoinError( @@ -1461,7 +1477,9 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { name: Seq[String], numMatches: Int, context: Origin): Throwable = { new AnalysisException( errorClass = "AMBIGUOUS_COLUMN_OR_FIELD", - messageParameters = Array(toSQLId(name), numMatches.toString), + messageParameters = Map( + "name" -> toSQLId(name), + "n" -> numMatches.toString), origin = context) } @@ -1469,7 +1487,9 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { name: Seq[String], numMatches: Int): Throwable = { new AnalysisException( errorClass = "AMBIGUOUS_COLUMN_OR_FIELD", - messageParameters = Array(toSQLId(name), numMatches.toString)) + messageParameters = Map( + "name" -> toSQLId(name), + "n" -> numMatches.toString)) } def cannotUseIntervalTypeInTableSchemaError(): Throwable = { @@ -1610,7 +1630,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { // The second argument of {function} function needs to be an integer new AnalysisException( errorClass = "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER", - messageParameters = Array(function), + messageParameters = Map("functionName" -> function), cause = Some(e)) } @@ -1663,7 +1683,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { new AnalysisException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "PYTHON_UDF_IN_ON_CLAUSE", - messageParameters = Array(s"${toSQLStmt(joinType.sql)}")) + messageParameters = Map("joinType" -> toSQLStmt(joinType.sql))) } def conflictingAttributesInJoinConditionError( @@ -2019,15 +2039,19 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def descPartitionNotAllowedOnTempView(table: String): Throwable = { new AnalysisException( errorClass = "FORBIDDEN_OPERATION", - messageParameters = - Array(toSQLStmt("DESC PARTITION"), "TEMPORARY VIEW", toSQLId(table))) + messageParameters = Map( + "statement" -> toSQLStmt("DESC PARTITION"), + "objectType" -> "TEMPORARY VIEW", + "objectName" -> toSQLId(table))) } def descPartitionNotAllowedOnView(table: String): Throwable = { new AnalysisException( errorClass = "FORBIDDEN_OPERATION", - messageParameters = Array( - toSQLStmt("DESC PARTITION"), "VIEW", toSQLId(table))) + messageParameters = Map( + "statement" -> toSQLStmt("DESC PARTITION"), + "objectType" -> "VIEW", + "objectName" -> toSQLId(table))) } def showPartitionNotAllowedOnTableNotPartitionedError(tableIdentWithDB: String): Throwable = { @@ -2348,7 +2372,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def usingUntypedScalaUDFError(): Throwable = { new AnalysisException( errorClass = "UNTYPED_SCALA_UDF", - messageParameters = Array.empty) + messageParameters = Map.empty) } def aggregationFunctionAppliedOnNonNumericColumnError(colName: String): Throwable = { @@ -2386,20 +2410,20 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def udfClassDoesNotImplementAnyUDFInterfaceError(className: String): Throwable = { new AnalysisException( errorClass = "NO_UDF_INTERFACE_ERROR", - messageParameters = Array(className)) + messageParameters = Map("className" -> className)) } def udfClassImplementMultiUDFInterfacesError(className: String): Throwable = { new AnalysisException( errorClass = "MULTI_UDF_INTERFACE_ERROR", - messageParameters = Array(className)) + messageParameters = Map("className" -> className)) } def udfClassWithTooManyTypeArgumentsError(n: Int): Throwable = { new AnalysisException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "TOO_MANY_TYPE_ARGUMENTS_FOR_UDF_CLASS", - messageParameters = Array(s"$n")) + messageParameters = Map("num" -> s"$n")) } def classWithoutPublicNonArgumentConstructorError(className: String): Throwable = { @@ -2428,14 +2452,16 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def invalidFieldName(fieldName: Seq[String], path: Seq[String], context: Origin): Throwable = { new AnalysisException( errorClass = "INVALID_FIELD_NAME", - messageParameters = Array(toSQLId(fieldName), toSQLId(path)), + messageParameters = Map( + "fieldName" -> toSQLId(fieldName), + "path" -> toSQLId(path)), origin = context) } def invalidJsonSchema(schema: DataType): Throwable = { new AnalysisException( errorClass = "INVALID_JSON_SCHEMA_MAP_TYPE", - messageParameters = Array(toSQLType(schema))) + messageParameters = Map("jsonSchema" -> toSQLType(schema))) } def tableIndexNotSupportedError(errorMessage: String): Throwable = { @@ -2537,13 +2563,13 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def nullableColumnOrFieldError(name: Seq[String]): Throwable = { new AnalysisException( errorClass = "NULLABLE_COLUMN_OR_FIELD", - messageParameters = Array(toSQLId(name))) + messageParameters = Map("name" -> toSQLId(name))) } def nullableArrayOrMapElementError(path: Seq[String]): Throwable = { new AnalysisException( errorClass = "NULLABLE_ARRAY_OR_MAP_ELEMENT", - messageParameters = Array(toSQLId(path))) + messageParameters = Map("columnPath" -> toSQLId(path))) } def invalidColumnOrFieldDataTypeError( @@ -2552,13 +2578,16 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { expected: DataType): Throwable = { new AnalysisException( errorClass = "INVALID_COLUMN_OR_FIELD_DATA_TYPE", - messageParameters = Array(toSQLId(name), toSQLType(dt), toSQLType(expected))) + messageParameters = Map( + "name" -> toSQLId(name), + "type" -> toSQLType(dt), + "expectedType" -> toSQLType(expected))) } def columnNotInGroupByClauseError(expression: Expression): Throwable = { new AnalysisException( errorClass = "COLUMN_NOT_IN_GROUP_BY_CLAUSE", - messageParameters = Array(toSQLExpr(expression)) + messageParameters = Map("expression" -> toSQLExpr(expression)) ) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index 662a10cf3acf9..2515d43cf8278 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -66,28 +66,25 @@ import org.apache.spark.util.CircularBuffer private[sql] object QueryExecutionErrors extends QueryErrorsBase { def cannotEvaluateExpressionError(expression: Expression): Throwable = { - new SparkUnsupportedOperationException(errorClass = "INTERNAL_ERROR", - messageParameters = Array(s"Cannot evaluate expression: $expression")) + SparkException.internalError(s"Cannot evaluate expression: $expression") } def cannotGenerateCodeForExpressionError(expression: Expression): Throwable = { - new SparkUnsupportedOperationException(errorClass = "INTERNAL_ERROR", - messageParameters = Array(s"Cannot generate code for expression: $expression")) + SparkException.internalError(s"Cannot generate code for expression: $expression") } def cannotTerminateGeneratorError(generator: UnresolvedGenerator): Throwable = { - new SparkUnsupportedOperationException(errorClass = "INTERNAL_ERROR", - messageParameters = Array(s"Cannot terminate expression: $generator")) + SparkException.internalError(s"Cannot terminate expression: $generator") } def castingCauseOverflowError(t: Any, from: DataType, to: DataType): ArithmeticException = { new SparkArithmeticException( errorClass = "CAST_OVERFLOW", - messageParameters = Array( - toSQLValue(t, from), - toSQLType(from), - toSQLType(to), - toSQLConf(SQLConf.ANSI_ENABLED.key)), + messageParameters = Map( + "value" -> toSQLValue(t, from), + "sourceType" -> toSQLType(from), + "targetType" -> toSQLType(to), + "ansiConfig" -> toSQLConf(SQLConf.ANSI_ENABLED.key)), context = Array.empty, summary = "") } @@ -98,10 +95,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { columnName: String): ArithmeticException = { new SparkArithmeticException( errorClass = "CAST_OVERFLOW_IN_TABLE_INSERT", - messageParameters = Array( - toSQLType(from), - toSQLType(to), - toSQLId(columnName)), + messageParameters = Map( + "sourceType" -> toSQLType(from), + "targetType" -> toSQLType(to), + "columnName" -> toSQLId(columnName)), context = Array.empty, summary = "" ) @@ -114,11 +111,11 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { context: SQLQueryContext = null): ArithmeticException = { new SparkArithmeticException( errorClass = "NUMERIC_VALUE_OUT_OF_RANGE", - messageParameters = Array( - value.toPlainString, - decimalPrecision.toString, - decimalScale.toString, - toSQLConf(SQLConf.ANSI_ENABLED.key)), + messageParameters = Map( + "value" -> value.toPlainString, + "precision" -> decimalPrecision.toString, + "scale" -> decimalScale.toString, + "config" -> toSQLConf(SQLConf.ANSI_ENABLED.key)), context = getQueryContext(context), summary = getSummary(context)) } @@ -130,11 +127,11 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { context: SQLQueryContext): Throwable = { new SparkDateTimeException( errorClass = "CAST_INVALID_INPUT", - messageParameters = Array( - toSQLValue(value, from), - toSQLType(from), - toSQLType(to), - toSQLConf(SQLConf.ANSI_ENABLED.key)), + messageParameters = Map( + "expression" -> toSQLValue(value, from), + "sourceType" -> toSQLType(from), + "targetType" -> toSQLType(to), + "ansiConfig" -> toSQLConf(SQLConf.ANSI_ENABLED.key)), context = getQueryContext(context), summary = getSummary(context)) } @@ -144,11 +141,11 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { context: SQLQueryContext): SparkRuntimeException = { new SparkRuntimeException( errorClass = "CAST_INVALID_INPUT", - messageParameters = Array( - toSQLValue(s, StringType), - toSQLType(StringType), - toSQLType(BooleanType), - toSQLConf(SQLConf.ANSI_ENABLED.key)), + messageParameters = Map( + "expression" -> toSQLValue(s, StringType), + "sourceType" -> toSQLType(StringType), + "targetType" -> toSQLType(BooleanType), + "ansiConfig" -> toSQLConf(SQLConf.ANSI_ENABLED.key)), context = getQueryContext(context), summary = getSummary(context)) } @@ -159,11 +156,11 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { context: SQLQueryContext): SparkNumberFormatException = { new SparkNumberFormatException( errorClass = "CAST_INVALID_INPUT", - messageParameters = Array( - toSQLValue(s, StringType), - toSQLType(StringType), - toSQLType(to), - toSQLConf(SQLConf.ANSI_ENABLED.key)), + messageParameters = Map( + "expression" -> toSQLValue(s, StringType), + "sourceType" -> toSQLType(StringType), + "targetType" -> toSQLType(to), + "ansiConfig" -> toSQLConf(SQLConf.ANSI_ENABLED.key)), context = getQueryContext(context), summary = getSummary(context)) } @@ -175,44 +172,58 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { hint: String): SparkIllegalArgumentException = { new SparkIllegalArgumentException( errorClass = "CONVERSION_INVALID_INPUT", - messageParameters = Array( - toSQLValue(s, StringType), - toSQLValue(fmt, StringType), - toSQLType(to), - toSQLId(hint))) + messageParameters = Map( + "str" -> toSQLValue(s, StringType), + "fmt" -> toSQLValue(fmt, StringType), + "targetType" -> toSQLType(to), + "suggestion" -> toSQLId(hint))) } def cannotCastFromNullTypeError(to: DataType): Throwable = { - new SparkException(errorClass = "CANNOT_CAST_DATATYPE", - messageParameters = Array(NullType.typeName, to.typeName), null) + new SparkException( + errorClass = "CANNOT_CAST_DATATYPE", + messageParameters = Map( + "sourceType" -> NullType.typeName, + "targetType" -> to.typeName), + cause = null) } def cannotCastError(from: DataType, to: DataType): Throwable = { - new SparkException(errorClass = "CANNOT_CAST_DATATYPE", - messageParameters = Array(from.typeName, to.typeName), null) + new SparkException( + errorClass = "CANNOT_CAST_DATATYPE", + messageParameters = Map( + "sourceType" -> from.typeName, + "targetType" -> to.typeName), + cause = null) } def cannotParseDecimalError(): Throwable = { new SparkRuntimeException( errorClass = "CANNOT_PARSE_DECIMAL", - messageParameters = Array.empty) + messageParameters = Map.empty) } def dataTypeUnsupportedError(dataType: String, failure: String): Throwable = { - new SparkIllegalArgumentException(errorClass = "UNSUPPORTED_DATATYPE", - messageParameters = Array(dataType + failure)) + new SparkIllegalArgumentException( + errorClass = "UNSUPPORTED_DATATYPE", + messageParameters = Map("typeName" -> (dataType + failure))) } def failedExecuteUserDefinedFunctionError(funcCls: String, inputTypes: String, outputType: String, e: Throwable): Throwable = { - new SparkException(errorClass = "FAILED_EXECUTE_UDF", - messageParameters = Array(funcCls, inputTypes, outputType), e) + new SparkException( + errorClass = "FAILED_EXECUTE_UDF", + messageParameters = Map( + "functionName" -> funcCls, + "signature" -> inputTypes, + "result" -> outputType), + cause = e) } def divideByZeroError(context: SQLQueryContext): ArithmeticException = { new SparkArithmeticException( errorClass = "DIVIDE_BY_ZERO", - messageParameters = Array(toSQLConf(SQLConf.ANSI_ENABLED.key)), + messageParameters = Map("config" -> toSQLConf(SQLConf.ANSI_ENABLED.key)), context = getQueryContext(context), summary = getSummary(context)) } @@ -220,7 +231,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { def intervalDividedByZeroError(context: SQLQueryContext): ArithmeticException = { new SparkArithmeticException( errorClass = "INTERVAL_DIVIDED_BY_ZERO", - messageParameters = Array.empty, + messageParameters = Map.empty, context = getQueryContext(context), summary = getSummary(context)) } @@ -231,10 +242,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { context: SQLQueryContext): ArrayIndexOutOfBoundsException = { new SparkArrayIndexOutOfBoundsException( errorClass = "INVALID_ARRAY_INDEX", - messageParameters = Array( - toSQLValue(index, IntegerType), - toSQLValue(numElements, IntegerType), - toSQLConf(SQLConf.ANSI_ENABLED.key)), + messageParameters = Map( + "indexValue" -> toSQLValue(index, IntegerType), + "arraySize" -> toSQLValue(numElements, IntegerType), + "ansiConfig" -> toSQLConf(SQLConf.ANSI_ENABLED.key)), context = getQueryContext(context), summary = getSummary(context)) } @@ -245,11 +256,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { context: SQLQueryContext): ArrayIndexOutOfBoundsException = { new SparkArrayIndexOutOfBoundsException( errorClass = "INVALID_ARRAY_INDEX_IN_ELEMENT_AT", - messageParameters = - Array( - toSQLValue(index, IntegerType), - toSQLValue(numElements, IntegerType), - toSQLConf(SQLConf.ANSI_ENABLED.key)), + messageParameters = Map( + "indexValue" -> toSQLValue(index, IntegerType), + "arraySize" -> toSQLValue(numElements, IntegerType), + "ansiConfig" -> toSQLConf(SQLConf.ANSI_ENABLED.key)), context = getQueryContext(context), summary = getSummary(context)) } @@ -258,7 +268,9 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { new SparkDateTimeException( errorClass = "INVALID_FRACTION_OF_SECOND", errorSubClass = None, - Array(toSQLConf(SQLConf.ANSI_ENABLED.key)), + messageParameters = Map( + "ansiConfig" -> toSQLConf(SQLConf.ANSI_ENABLED.key) + ), context = Array.empty, summary = "") } @@ -267,7 +279,9 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { new SparkDateTimeException( errorClass = "CANNOT_PARSE_TIMESTAMP", errorSubClass = None, - Array(e.getMessage, toSQLConf(SQLConf.ANSI_ENABLED.key)), + messageParameters = Map( + "message" -> e.getMessage, + "ansiConfig" -> toSQLConf(SQLConf.ANSI_ENABLED.key)), context = Array.empty, summary = "") } @@ -306,14 +320,18 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { new SparkRuntimeException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "LITERAL_TYPE", - messageParameters = Array( s"${v.toString}", s"${v.getClass.toString}")) + messageParameters = Map( + "value" -> v.toString, + "type" -> v.getClass.toString)) } def pivotColumnUnsupportedError(v: Any, dataType: DataType): RuntimeException = { new SparkRuntimeException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "PIVOT_TYPE", - messageParameters = Array(s"${v.toString}", s"${toSQLType(dataType)}")) + messageParameters = Map( + "value" -> v.toString, + "type" -> toSQLType(dataType))) } def noDefaultForDataTypeError(dataType: DataType): RuntimeException = { @@ -340,8 +358,9 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { } def illegalUrlError(url: UTF8String): Throwable = { - new SparkIllegalArgumentException(errorClass = "CANNOT_DECODE_URL", - messageParameters = Array(url.toString) + new SparkIllegalArgumentException( + errorClass = "CANNOT_DECODE_URL", + messageParameters = Map("url" -> url.toString) ) } @@ -394,9 +413,8 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { } def methodNotDeclaredError(name: String): Throwable = { - new SparkNoSuchMethodException(errorClass = "INTERNAL_ERROR", - messageParameters = Array( - s"""A method named "$name" is not declared in any enclosing class nor any supertype""")) + SparkException.internalError( + s"""A method named "$name" is not declared in any enclosing class nor any supertype""") } def constructorNotFoundError(cls: String): Throwable = { @@ -516,7 +534,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { } else "" new SparkArithmeticException( errorClass = "ARITHMETIC_OVERFLOW", - messageParameters = Array(message, alternative, SQLConf.ANSI_ENABLED.key), + messageParameters = Map( + "message" -> message, + "alternative" -> alternative, + "config" -> toSQLConf(SQLConf.ANSI_ENABLED.key)), context = getQueryContext(context), summary = getSummary(context)) } @@ -540,7 +561,9 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { } else "" new SparkArithmeticException( errorClass = "INTERVAL_ARITHMETIC_OVERFLOW", - messageParameters = Array(message, alternative), + messageParameters = Map( + "message" -> message, + "alternative" -> alternative), context = getQueryContext(context), summary = getSummary(context)) } @@ -579,10 +602,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { """.stripMargin) } - def inferDateWithLegacyTimeParserError(): Throwable with SparkThrowable = { - new SparkIllegalArgumentException(errorClass = "CANNOT_INFER_DATE", - messageParameters = Array() - ) + def inferDateWithLegacyTimeParserError(): Throwable = { + new SparkIllegalArgumentException( + errorClass = "CANNOT_INFER_DATE", + messageParameters = Map.empty) } def streamedOperatorUnsupportedByDataSourceError( @@ -610,8 +633,11 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { } def incompatibleDataSourceRegisterError(e: Throwable): Throwable = { - new SparkClassNotFoundException("INCOMPATIBLE_DATASOURCE_REGISTER", None, - Array(e.getMessage), e) + new SparkClassNotFoundException( + errorClass = "INCOMPATIBLE_DATASOURCE_REGISTER", + errorSubClass = None, + messageParameters = Map("message" -> e.getMessage), + cause = e) } def sparkUpgradeInReadingDatesError( @@ -619,12 +645,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { new SparkUpgradeException( errorClass = "INCONSISTENT_BEHAVIOR_CROSS_VERSION", errorSubClass = Some("READ_ANCIENT_DATETIME"), - messageParameters = Array( - format, - toSQLConf(config), - toDSOption(option), - toSQLConf(config), - toDSOption(option)), + messageParameters = Map( + "format" -> format, + "config" -> toSQLConf(config), + "option" -> toDSOption(option)), cause = null ) } @@ -633,10 +657,9 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { new SparkUpgradeException( errorClass = "INCONSISTENT_BEHAVIOR_CROSS_VERSION", errorSubClass = Some("WRITE_ANCIENT_DATETIME"), - messageParameters = Array( - format, - toSQLConf(config), - toSQLConf(config)), + messageParameters = Map( + "format" -> format, + "config" -> toSQLConf(config)), cause = null ) } @@ -664,14 +687,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { } def saveModeUnsupportedError(saveMode: Any, pathExists: Boolean): Throwable = { - pathExists match { - case true => new SparkIllegalArgumentException(errorClass = "UNSUPPORTED_SAVE_MODE", - errorSubClass = Some("EXISTENT_PATH"), - messageParameters = Array(toSQLValue(saveMode, StringType))) - case _ => new SparkIllegalArgumentException(errorClass = "UNSUPPORTED_SAVE_MODE", - errorSubClass = Some("NON_EXISTENT_PATH"), - messageParameters = Array(toSQLValue(saveMode, StringType))) - } + new SparkIllegalArgumentException( + errorClass = "UNSUPPORTED_SAVE_MODE", + errorSubClass = Some(if (pathExists) "EXISTENT_PATH" else "NON_EXISTENT_PATH"), + messageParameters = Map("saveMode" -> toSQLValue(saveMode, StringType))) } def cannotClearOutputDirectoryError(staticPrefixPath: Path): Throwable = { @@ -759,7 +778,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { def writingJobAbortedError(e: Throwable): Throwable = { new SparkException( errorClass = "WRITING_JOB_ABORTED", - messageParameters = Array.empty, + messageParameters = Map.empty, cause = e) } @@ -825,7 +844,9 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { } def unrecognizedSqlTypeError(sqlType: Int): Throwable = { - new SparkSQLException(errorClass = "UNRECOGNIZED_SQL_TYPE", Array(sqlType.toString)) + new SparkSQLException( + errorClass = "UNRECOGNIZED_SQL_TYPE", + messageParameters = Map("typeName" -> sqlType.toString)) } def unsupportedJdbcTypeError(content: String): Throwable = { @@ -855,7 +876,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { new SparkSQLFeatureNotSupportedException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "JDBC_TRANSACTION", - messageParameters = Array[String]()) + messageParameters = Map.empty[String, String]) } def dataTypeUnsupportedYetError(dataType: DataType): Throwable = { @@ -1062,9 +1083,9 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { new SparkUpgradeException( errorClass = "INCONSISTENT_BEHAVIOR_CROSS_VERSION", errorSubClass = Some("PARSE_DATETIME_BY_NEW_PARSER"), - messageParameters = Array( - toSQLValue(s, StringType), - toSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key)), + messageParameters = Map( + "datetime" -> toSQLValue(s, StringType), + "config" -> toSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key)), e) } @@ -1072,9 +1093,9 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { new SparkUpgradeException( errorClass = "INCONSISTENT_BEHAVIOR_CROSS_VERSION", errorSubClass = Some("DATETIME_PATTERN_RECOGNITION"), - messageParameters = Array( - toSQLValue(pattern, StringType), - toSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key)), + messageParameters = Map( + "pattern" -> toSQLValue(pattern, StringType), + "config" -> toSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key)), e) } @@ -1090,7 +1111,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { } def concurrentQueryInstanceError(): Throwable = { - new SparkConcurrentModificationException("CONCURRENT_QUERY", None, Array.empty) + new SparkConcurrentModificationException( + errorClass = "CONCURRENT_QUERY", + errorSubClass = None, + messageParameters = Map.empty[String, String]) } def cannotParseJsonArraysAsStructsError(): Throwable = { @@ -1236,7 +1260,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { new SparkRuntimeException( errorClass = "ELEMENT_AT_BY_INDEX_ZERO", cause = null, - messageParameters = Array.empty, + messageParameters = Map.empty, context = getQueryContext(context), summary = getSummary(context)) } @@ -1490,8 +1514,12 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { } def renamePathAsExistsPathError(srcPath: Path, dstPath: Path): Throwable = { - new SparkFileAlreadyExistsException(errorClass = "FAILED_RENAME_PATH", None, - Array(srcPath.toString, dstPath.toString)) + new SparkFileAlreadyExistsException( + errorClass = "FAILED_RENAME_PATH", + errorSubClass = None, + messageParameters = Map( + "sourcePath" -> srcPath.toString, + "targetPath" -> dstPath.toString)) } def renameAsExistsPathError(dstPath: Path): Throwable = { @@ -1499,8 +1527,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { } def renameSrcPathNotFoundError(srcPath: Path): Throwable = { - new SparkFileNotFoundException(errorClass = "RENAME_SRC_PATH_NOT_FOUND", None, - Array(srcPath.toString)) + new SparkFileNotFoundException( + errorClass = "RENAME_SRC_PATH_NOT_FOUND", + errorSubClass = None, + messageParameters = Map("sourcePath" -> srcPath.toString)) } def failedRenameTempFileError(srcPath: Path, dstPath: Path): Throwable = { @@ -1697,8 +1727,12 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { permission: FsPermission, path: Path, e: Throwable): Throwable = { - new SparkSecurityException(errorClass = "RESET_PERMISSION_TO_ORIGINAL", None, - Array(permission.toString, path.toString, e.getMessage)) + new SparkSecurityException( + errorClass = "RESET_PERMISSION_TO_ORIGINAL", None, + messageParameters = Map( + "permission" -> permission.toString, + "path" -> path.toString, + "message" -> e.getMessage)) } def failToSetOriginalACLBackError(aclEntries: String, path: Path, e: Throwable): Throwable = { @@ -1952,14 +1986,14 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { new SparkUnsupportedOperationException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "REPEATED_PIVOT", - messageParameters = Array[String]()) + messageParameters = Map.empty[String, String]) } def pivotNotAfterGroupByUnsupportedError(): Throwable = { new SparkUnsupportedOperationException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "PIVOT_AFTER_GROUP_BY", - messageParameters = Array[String]()) + messageParameters = Map.empty[String, String]) } private val aesFuncName = toSQLId("aes_encrypt") + "/" + toSQLId("aes_decrypt") @@ -1967,26 +2001,30 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { def invalidAesKeyLengthError(actualLength: Int): RuntimeException = { new SparkRuntimeException( errorClass = "INVALID_PARAMETER_VALUE", - messageParameters = Array( - "key", - aesFuncName, - s"expects a binary value with 16, 24 or 32 bytes, but got ${actualLength.toString} bytes.")) + messageParameters = Map( + "parameter" -> "key", + "functionName" -> aesFuncName, + "expected" -> ("expects a binary value with 16, 24 or 32 bytes, " + + s"but got ${actualLength.toString} bytes."))) } def aesModeUnsupportedError(mode: String, padding: String): RuntimeException = { new SparkRuntimeException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "AES_MODE", - messageParameters = Array(mode, padding, aesFuncName)) + messageParameters = Map( + "mode" -> mode, + "padding" -> padding, + "functionName" -> aesFuncName)) } def aesCryptoError(detailMessage: String): RuntimeException = { new SparkRuntimeException( errorClass = "INVALID_PARAMETER_VALUE", - messageParameters = Array( - "expr, key", - aesFuncName, - s"Detail message: $detailMessage")) + messageParameters = Map( + "parameter" -> "expr, key", + "functionName" -> aesFuncName, + "expected" -> s"Detail message: $detailMessage")) } def hiveTableWithAnsiIntervalsError(tableName: String): Throwable = { @@ -1997,16 +2035,18 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { new SparkUnsupportedOperationException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "ORC_TYPE_CAST", - messageParameters = Array(toSQLType(TimestampType), - toSQLType(TimestampNTZType))) + messageParameters = Map( + "orcType" -> toSQLType(TimestampType), + "toType" -> toSQLType(TimestampNTZType))) } def cannotConvertOrcTimestampNTZToTimestampLTZError(): Throwable = { new SparkUnsupportedOperationException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "ORC_TYPE_CAST", - messageParameters = Array(toSQLType(TimestampNTZType), - toSQLType(TimestampType))) + messageParameters = Map( + "orcType" -> toSQLType(TimestampNTZType), + "toType" -> toSQLType(TimestampType))) } def writePartitionExceedConfigSizeWhenDynamicPartitionError( @@ -2044,38 +2084,52 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { def timestampAddOverflowError(micros: Long, amount: Int, unit: String): ArithmeticException = { new SparkArithmeticException( errorClass = "DATETIME_OVERFLOW", - messageParameters = Array( - s"add ${toSQLValue(amount, IntegerType)} $unit to " + - s"${toSQLValue(DateTimeUtils.microsToInstant(micros), TimestampType)}"), + messageParameters = Map( + "operation" -> (s"add ${toSQLValue(amount, IntegerType)} $unit to " + + s"${toSQLValue(DateTimeUtils.microsToInstant(micros), TimestampType)}")), context = Array.empty, summary = "") } def invalidBucketFile(path: String): Throwable = { - new SparkException(errorClass = "INVALID_BUCKET_FILE", messageParameters = Array(path), + new SparkException( + errorClass = "INVALID_BUCKET_FILE", + messageParameters = Map("path" -> path), cause = null) } def multipleRowSubqueryError(context: SQLQueryContext): Throwable = { new SparkException( errorClass = "MULTI_VALUE_SUBQUERY_ERROR", - messageParameters = Array.empty, + messageParameters = Map.empty, cause = null, context = getQueryContext(context), summary = getSummary(context)) } def nullComparisonResultError(): Throwable = { - new SparkException(errorClass = "NULL_COMPARISON_RESULT", - messageParameters = Array(), cause = null) + new SparkException( + errorClass = "NULL_COMPARISON_RESULT", + messageParameters = Map.empty, + cause = null) } def invalidPatternError(funcName: String, pattern: String): RuntimeException = { new SparkRuntimeException( errorClass = "INVALID_PARAMETER_VALUE", - messageParameters = Array( - "regexp", - toSQLId(funcName), - pattern)) + messageParameters = Map( + "parameter" -> "regexp", + "functionName" -> toSQLId(funcName), + "expected" -> pattern)) + } + + def tooManyArrayElementsError( + numElements: Int, + elementSize: Int): SparkIllegalArgumentException = { + new SparkIllegalArgumentException( + errorClass = "TOO_MANY_ARRAY_ELEMENTS", + messageParameters = Map( + "numElements" -> numElements.toString, + "size" -> elementSize.toString)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala index d4629f0dd3fe4..51b68383cd0e4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala @@ -79,8 +79,8 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { def emptyPartitionKeyError(key: String, ctx: PartitionSpecContext): Throwable = { new ParseException( errorClass = "INVALID_SQL_SYNTAX", - messageParameters = - Array(s"Partition key ${toSQLId(key)} must set value (can't be empty)."), + messageParameters = Map( + "inputString" -> s"Partition key ${toSQLId(key)} must set value (can't be empty)."), ctx) } @@ -97,7 +97,7 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { new ParseException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "TRANSFORM_DISTINCT_ALL", - messageParameters = Array[String](), + messageParameters = Map.empty, ctx) } @@ -105,7 +105,7 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { new ParseException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "TRANSFORM_NON_HIVE", - messageParameters = Array[String](), + messageParameters = Map.empty, ctx) } @@ -117,7 +117,7 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { new ParseException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "LATERAL_NATURAL_JOIN", - messageParameters = Array[String](), + messageParameters = Map.empty, ctx) } @@ -125,7 +125,7 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { new ParseException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "LATERAL_JOIN_USING", - messageParameters = Array[String](), + messageParameters = Map.empty, ctx) } @@ -133,36 +133,47 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { new ParseException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "LATERAL_JOIN_OF_TYPE", - messageParameters = Array(s"${toSQLStmt(joinType)}"), + messageParameters = Map("joinType" -> toSQLStmt(joinType)), ctx) } def invalidLateralJoinRelationError(ctx: RelationPrimaryContext): Throwable = { new ParseException( errorClass = "INVALID_SQL_SYNTAX", - messageParameters = Array(s"${toSQLStmt("LATERAL")} can only be used with subquery."), + messageParameters = Map( + "inputString" -> s"${toSQLStmt("LATERAL")} can only be used with subquery."), ctx) } def repetitiveWindowDefinitionError(name: String, ctx: WindowClauseContext): Throwable = { - new ParseException("INVALID_SQL_SYNTAX", - Array(s"The definition of window ${toSQLId(name)} is repetitive."), ctx) + new ParseException( + errorClass = "INVALID_SQL_SYNTAX", + messageParameters = Map( + "inputString" -> s"The definition of window ${toSQLId(name)} is repetitive."), + ctx) } def invalidWindowReferenceError(name: String, ctx: WindowClauseContext): Throwable = { - new ParseException("INVALID_SQL_SYNTAX", - Array(s"Window reference ${toSQLId(name)} is not a window specification."), ctx) + new ParseException( + errorClass = "INVALID_SQL_SYNTAX", + messageParameters = Map( + "inputString" -> s"Window reference ${toSQLId(name)} is not a window specification."), + ctx) } def cannotResolveWindowReferenceError(name: String, ctx: WindowClauseContext): Throwable = { - new ParseException("INVALID_SQL_SYNTAX", - Array(s"Cannot resolve window reference ${toSQLId(name)}."), ctx) + new ParseException( + errorClass = "INVALID_SQL_SYNTAX", + messageParameters = Map( + "inputString" -> s"Cannot resolve window reference ${toSQLId(name)}."), + ctx) } def naturalCrossJoinUnsupportedError(ctx: RelationContext): Throwable = { - new ParseException(errorClass = "UNSUPPORTED_FEATURE", + new ParseException( + errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "NATURAL_CROSS_JOIN", - messageParameters = Array[String](), + messageParameters = Map.empty, ctx = ctx) } @@ -189,8 +200,11 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { } def functionNameUnsupportedError(functionName: String, ctx: ParserRuleContext): Throwable = { - new ParseException("INVALID_SQL_SYNTAX", - Array(s"Unsupported function name ${toSQLId(functionName)}"), ctx) + new ParseException( + errorClass = "INVALID_SQL_SYNTAX", + messageParameters = Map( + "inputString" -> s"Unsupported function name ${toSQLId(functionName)}"), + ctx) } def cannotParseValueTypeError( @@ -250,22 +264,27 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { } def charTypeMissingLengthError(dataType: String, ctx: PrimitiveDataTypeContext): Throwable = { - new ParseException("PARSE_CHAR_MISSING_LENGTH", Array(dataType, dataType), ctx) + new ParseException( + errorClass = "PARSE_CHAR_MISSING_LENGTH", + messageParameters = Map("type" -> toSQLType(dataType)), + ctx) } def partitionTransformNotExpectedError( name: String, describe: String, ctx: ApplyTransformContext): Throwable = { new ParseException( errorClass = "INVALID_SQL_SYNTAX", - messageParameters = - Array(s"Expected a column reference for transform ${toSQLId(name)}: $describe"), + messageParameters = Map( + "inputString" -> + s"Expected a column reference for transform ${toSQLId(name)}: $describe"), ctx) } def tooManyArgumentsForTransformError(name: String, ctx: ApplyTransformContext): Throwable = { new ParseException( errorClass = "INVALID_SQL_SYNTAX", - messageParameters = Array(s"Too many arguments for transform ${toSQLId(name)}"), + messageParameters = Map( + "inputString" -> s"Too many arguments for transform ${toSQLId(name)}"), ctx) } @@ -278,7 +297,7 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { new ParseException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "SET_NAMESPACE_PROPERTY", - messageParameters = Array(property, msg), + messageParameters = Map("property" -> property, "msg" -> msg), ctx) } @@ -286,7 +305,7 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { new ParseException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "SET_PROPERTIES_AND_DBPROPERTIES", - messageParameters = Array[String](), + messageParameters = Map.empty, ctx ) } @@ -296,7 +315,7 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { new ParseException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "SET_TABLE_PROPERTY", - messageParameters = Array(property, msg), + messageParameters = Map("property" -> property, "msg" -> msg), ctx) } @@ -330,7 +349,7 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { new ParseException( errorClass = "UNSUPPORTED_FEATURE", errorSubClass = "DESC_TABLE_COLUMN_PARTITION", - messageParameters = Array[String](), + messageParameters = Map.empty, ctx) } @@ -338,7 +357,8 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { key: String, ctx: DescribeRelationContext): Throwable = { new ParseException( errorClass = "INVALID_SQL_SYNTAX", - messageParameters = Array(s"PARTITION specification is incomplete: ${toSQLId(key)}"), + messageParameters = Map( + "inputString" -> s"PARTITION specification is incomplete: ${toSQLId(key)}"), ctx) } @@ -355,17 +375,19 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { def showFunctionsUnsupportedError(identifier: String, ctx: IdentifierContext): Throwable = { new ParseException( errorClass = "INVALID_SQL_SYNTAX", - messageParameters = Array( - s"${toSQLStmt("SHOW")} ${toSQLId(identifier)} ${toSQLStmt("FUNCTIONS")} not supported"), + messageParameters = Map( + "inputString" -> + s"${toSQLStmt("SHOW")} ${toSQLId(identifier)} ${toSQLStmt("FUNCTIONS")} not supported"), ctx) } def showFunctionsInvalidPatternError(pattern: String, ctx: ParserRuleContext): Throwable = { new ParseException( errorClass = "INVALID_SQL_SYNTAX", - messageParameters = Array( - s"Invalid pattern in ${toSQLStmt("SHOW FUNCTIONS")}: ${toSQLId(pattern)}. " + - s"It must be a ${toSQLType(StringType)} literal."), + messageParameters = Map( + "inputString" -> + (s"Invalid pattern in ${toSQLStmt("SHOW FUNCTIONS")}: ${toSQLId(pattern)}. " + + s"It must be a ${toSQLType(StringType)} literal.")), ctx) } @@ -388,7 +410,10 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { def duplicateKeysError(key: String, ctx: ParserRuleContext): Throwable = { // Found duplicate keys '$key' - new ParseException(errorClass = "DUPLICATE_KEY", messageParameters = Array(toSQLId(key)), ctx) + new ParseException( + errorClass = "DUPLICATE_KEY", + messageParameters = Map("keyColumn" -> toSQLId(key)), + ctx) } def unexpectedFormatForSetConfigurationError(ctx: ParserRuleContext): Throwable = { @@ -400,17 +425,21 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { def invalidPropertyKeyForSetQuotedConfigurationError( keyCandidate: String, valueStr: String, ctx: ParserRuleContext): Throwable = { - new ParseException(errorClass = "INVALID_PROPERTY_KEY", - messageParameters = Array(toSQLConf(keyCandidate), - toSQLConf(keyCandidate), toSQLConf(valueStr)), + new ParseException( + errorClass = "INVALID_PROPERTY_KEY", + messageParameters = Map( + "key" -> toSQLConf(keyCandidate), + "value" -> toSQLConf(valueStr)), ctx) } def invalidPropertyValueForSetQuotedConfigurationError( valueCandidate: String, keyStr: String, ctx: ParserRuleContext): Throwable = { - new ParseException(errorClass = "INVALID_PROPERTY_VALUE", - messageParameters = Array(toSQLConf(valueCandidate), - toSQLConf(keyStr), toSQLConf(valueCandidate)), + new ParseException( + errorClass = "INVALID_PROPERTY_VALUE", + messageParameters = Map( + "value" -> toSQLConf(valueCandidate), + "key" -> toSQLConf(keyStr)), ctx) } @@ -473,33 +502,40 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { def createFuncWithBothIfNotExistsAndReplaceError(ctx: CreateFunctionContext): Throwable = { new ParseException( errorClass = "INVALID_SQL_SYNTAX", - messageParameters = Array( - s"${toSQLStmt("CREATE FUNCTION")} with both ${toSQLStmt("IF NOT EXISTS")} " + - s"and ${toSQLStmt("REPLACE")} is not allowed."), + messageParameters = Map( + "inputString" -> + (s"${toSQLStmt("CREATE FUNCTION")} with both ${toSQLStmt("IF NOT EXISTS")} " + + s"and ${toSQLStmt("REPLACE")} is not allowed.")), ctx) } def defineTempFuncWithIfNotExistsError(ctx: CreateFunctionContext): Throwable = { new ParseException( errorClass = "INVALID_SQL_SYNTAX", - messageParameters = Array( - s"It is not allowed to define a ${toSQLStmt("TEMPORARY FUNCTION")}" + - s" with ${toSQLStmt("IF NOT EXISTS")}."), ctx) + messageParameters = Map( + "inputString" -> + (s"It is not allowed to define a ${toSQLStmt("TEMPORARY FUNCTION")}" + + s" with ${toSQLStmt("IF NOT EXISTS")}.")), + ctx) } def unsupportedFunctionNameError(funcName: Seq[String], ctx: CreateFunctionContext): Throwable = { - new ParseException("INVALID_SQL_SYNTAX", - Array(s"Unsupported function name ${toSQLId(funcName)}"), ctx) + new ParseException( + errorClass = "INVALID_SQL_SYNTAX", + messageParameters = Map( + "inputString" -> s"Unsupported function name ${toSQLId(funcName)}"), + ctx) } def specifyingDBInCreateTempFuncError( databaseName: String, ctx: CreateFunctionContext): Throwable = { new ParseException( - "INVALID_SQL_SYNTAX", - Array( - s"Specifying a database in ${toSQLStmt("CREATE TEMPORARY FUNCTION")} is not allowed: " + - toSQLId(databaseName)), + errorClass = "INVALID_SQL_SYNTAX", + messageParameters = Map( + "inputString" -> + (s"Specifying a database in ${toSQLStmt("CREATE TEMPORARY FUNCTION")} is not allowed: " + + toSQLId(databaseName))), ctx) } @@ -507,8 +543,11 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { name: Seq[String], ctx: TableValuedFunctionContext): Throwable = { new ParseException( - "INVALID_SQL_SYNTAX", - Array("table valued function cannot specify database name ", toSQLId(name)), ctx) + errorClass = "INVALID_SQL_SYNTAX", + messageParameters = Map( + "inputString" -> + ("table valued function cannot specify database name: " + toSQLId(name))), + ctx) } def unclosedBracketedCommentError(command: String, position: Origin): Throwable = { @@ -522,9 +561,10 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { def invalidNameForDropTempFunc(name: Seq[String], ctx: ParserRuleContext): Throwable = { new ParseException( errorClass = "INVALID_SQL_SYNTAX", - messageParameters = Array( - s"${toSQLStmt("DROP TEMPORARY FUNCTION")} requires a single part name but got: " + - toSQLId(name)), + messageParameters = Map( + "inputString" -> + (s"${toSQLStmt("DROP TEMPORARY FUNCTION")} requires a single part name but got: " + + toSQLId(name))), ctx) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 553f46a00c27c..29c28f85e0ee0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -116,7 +116,7 @@ class AnalysisErrorSuite extends AnalysisTest { name: String, plan: LogicalPlan, errorClass: String, - messageParameters: Array[String]): Unit = { + messageParameters: Map[String, String]): Unit = { errorClassTest(name, plan, errorClass, null, messageParameters) } @@ -125,7 +125,7 @@ class AnalysisErrorSuite extends AnalysisTest { plan: LogicalPlan, errorClass: String, errorSubClass: String, - messageParameters: Array[String]): Unit = { + messageParameters: Map[String, String]): Unit = { test(name) { assertAnalysisErrorClass(plan, errorClass, errorSubClass, messageParameters, caseSensitive = true) @@ -303,7 +303,7 @@ class AnalysisErrorSuite extends AnalysisTest { testRelation.select($"abcd"), "UNRESOLVED_COLUMN", "WITH_SUGGESTION", - Array("`abcd`", "`a`")) + Map("objectName" -> "`abcd`", "proposal" -> "`a`")) errorClassTest( "unresolved attributes with a generated name", @@ -312,7 +312,7 @@ class AnalysisErrorSuite extends AnalysisTest { .orderBy($"havingCondition".asc), "UNRESOLVED_COLUMN", "WITH_SUGGESTION", - Array("`havingCondition`", "`max(b)`")) + Map("objectName" -> "`havingCondition`", "proposal" -> "`max(b)`")) errorTest( "unresolved star expansion in max", @@ -329,7 +329,7 @@ class AnalysisErrorSuite extends AnalysisTest { testRelation2.groupBy($"a", $"c")($"a", $"c", count($"a").as("a3")).orderBy($"b".asc), "UNRESOLVED_COLUMN", "WITH_SUGGESTION", - Array("`b`", "`a`, `c`, `a3`")) + Map("objectName" -> "`b`", "proposal" -> "`a`, `c`, `a3`")) errorTest( "non-boolean filters", @@ -424,7 +424,7 @@ class AnalysisErrorSuite extends AnalysisTest { testRelation2.where($"bad_column" > 1).groupBy($"a")(UnresolvedAlias(max($"b"))), "UNRESOLVED_COLUMN", "WITH_SUGGESTION", - Array("`bad_column`", "`a`, `b`, `c`, `d`, `e`")) + Map("objectName" -> "`bad_column`", "proposal" -> "`a`, `b`, `c`, `d`, `e`")) errorTest( "slide duration greater than window in time window", diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 1fd001c27d615..e161a61cdf92a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -105,7 +105,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))), "UNRESOLVED_COLUMN", "WITH_SUGGESTION", - Array("`tBl`.`a`", "`TbL`.`a`"), + Map("objectName" -> "`tBl`.`a`", "proposal" -> "`TbL`.`a`"), caseSensitive = true) checkAnalysisWithoutViewWrapper( @@ -715,7 +715,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { assertAnalysisErrorClass(parsePlan("WITH t(x) AS (SELECT 1) SELECT * FROM t WHERE y = 1"), "UNRESOLVED_COLUMN", "WITH_SUGGESTION", - Array("`y`", "`t`.`x`"), + Map("objectName" -> "`y`", "proposal" -> "`t`.`x`"), caseSensitive = true) } @@ -1155,7 +1155,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { |""".stripMargin), "UNRESOLVED_COLUMN", "WITH_SUGGESTION", - Array("`c`.`y`", "`x`"), + Map("objectName" -> "`c`.`y`", "proposal" -> "`x`"), caseSensitive = true) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala index 94cb68a26f75b..8bb0588b536e6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala @@ -173,7 +173,7 @@ trait AnalysisTest extends PlanTest { protected def assertAnalysisErrorClass( inputPlan: LogicalPlan, expectedErrorClass: String, - expectedMessageParameters: Array[String], + expectedMessageParameters: Map[String, String], caseSensitive: Boolean = true): Unit = { assertAnalysisErrorClass( inputPlan, @@ -187,7 +187,7 @@ trait AnalysisTest extends PlanTest { inputPlan: LogicalPlan, expectedErrorClass: String, expectedErrorSubClass: String, - expectedMessageParameters: Array[String], + expectedMessageParameters: Map[String, String], caseSensitive: Boolean): Unit = { withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { val analyzer = getAnalyzer diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala index b3a19041220b2..ab4d79593e7b6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala @@ -135,7 +135,7 @@ class ResolveSubquerySuite extends AnalysisTest { lateralJoin(t1, lateralJoin(t2, t0.select($"a", $"b", $"c"))), "UNRESOLVED_COLUMN", "WITHOUT_SUGGESTION", - Array("`a`"), + Map("objectName" -> "`a`"), caseSensitive = true) } @@ -145,28 +145,28 @@ class ResolveSubquerySuite extends AnalysisTest { lateralJoin(t1, t0.select($"a", $"c")), "UNRESOLVED_COLUMN", "WITHOUT_SUGGESTION", - Array("`c`"), + Map("objectName" -> "`c`"), caseSensitive = true) // SELECT * FROM t1, LATERAL (SELECT a, b, c, d FROM t2) assertAnalysisErrorClass( lateralJoin(t1, t2.select($"a", $"b", $"c", $"d")), "UNRESOLVED_COLUMN", "WITH_SUGGESTION", - Array("`d`", "`b`, `c`"), + Map("objectName" -> "`d`", "proposal" -> "`b`, `c`"), caseSensitive = true) // SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.a)) assertAnalysisErrorClass( lateralJoin(t1, lateralJoin(t2, t0.select($"t1.a"))), "UNRESOLVED_COLUMN", "WITHOUT_SUGGESTION", - Array("`t1`.`a`"), + Map("objectName" -> "`t1`.`a`"), caseSensitive = true) // SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT a, b)) assertAnalysisErrorClass( lateralJoin(t1, lateralJoin(t2, t0.select($"a", $"b"))), "UNRESOLVED_COLUMN", "WITHOUT_SUGGESTION", - Array("`a`"), + Map("objectName" -> "`a`"), caseSensitive = true) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala index e5fda40cf516c..d5c9660c77a80 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala @@ -692,7 +692,7 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { parsedPlan, "UNRESOLVED_COLUMN", "WITH_SUGGESTION", - Array("`a`", "`x`, `y`"), + Map("objectName" -> "`a`", "proposal" -> "`x`, `y`"), caseSensitive = true) val tableAcceptAnySchema = TestRelationAcceptAnySchema(StructType(Seq( @@ -706,7 +706,7 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { parsedPlan2, "UNRESOLVED_COLUMN", "WITH_SUGGESTION", - Array("`a`", "`x`, `y`"), + Map("objectName" -> "`a`", "proposal" -> "`x`, `y`"), caseSensitive = true) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala index d814236c3fe94..43b7f35f7bb24 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala @@ -183,4 +183,11 @@ class CanonicalizeSuite extends SparkFunSuite { // canonicalization should not converted resolved cast to unresolved assert(cast.canonicalized.resolved) } + + test("SPARK-40362: Commutative operator under BinaryComparison") { + Seq(EqualTo, EqualNullSafe, GreaterThan, LessThan, GreaterThanOrEqual, LessThanOrEqual) + .foreach { bc => + assert(bc(Add($"a", $"b"), Literal(10)).semanticEquals(bc(Add($"b", $"a"), Literal(10)))) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala index 08fd53b353497..b16629f59aa2d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala @@ -459,7 +459,7 @@ case class CodegenFallbackExpression(child: Expression) } case class ProxyExpression(child: Expression) extends UnaryExpression { - override lazy val preCanonicalized: Expression = child.preCanonicalized + override lazy val canonicalized: Expression = child.canonicalized override def dataType: DataType = child.dataType override def eval(input: InternalRow): Any = child.eval(input) override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala index faa8e6fb0bfed..781c1c20783cb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala @@ -21,7 +21,7 @@ import scala.reflect.ClassTag import org.scalatest.PrivateMethodTester -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.internal.SQLConf @@ -30,7 +30,7 @@ import org.apache.spark.sql.types.{LongType, StructField, StructType, TimestampN class TimeWindowSuite extends SparkFunSuite with ExpressionEvalHelper with PrivateMethodTester { test("time window is unevaluable") { - intercept[UnsupportedOperationException] { + intercept[SparkException] { evaluateWithoutCodegen(TimeWindow(Literal(10L), "1 second", "1 second", "0 second")) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriterSuite.scala new file mode 100644 index 0000000000000..447454603001e --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriterSuite.scala @@ -0,0 +1,31 @@ +/* + * 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.expressions.codegen + +import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException} + +class UnsafeArrayWriterSuite extends SparkFunSuite { + test("SPARK-40403: don't print negative number when array is too big") { + val rowWriter = new UnsafeRowWriter(1) + rowWriter.resetRowWriter() + val arrayWriter = new UnsafeArrayWriter(rowWriter, 8) + assert(intercept[SparkIllegalArgumentException] { + arrayWriter.initialize(268271216) + }.getMessage.contains("Cannot initialize array with 268271216 elements of size 8")) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala index 52d0c6c701820..b99fd089f0ae4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala @@ -215,10 +215,10 @@ class ErrorParserSuite extends AnalysisTest { // special handling on char and varchar intercept("SELECT cast('a' as CHAR)", "PARSE_CHAR_MISSING_LENGTH", 1, 19, 19, - "DataType char requires a length parameter") + "DataType \"CHAR\" requires a length parameter") intercept("SELECT cast('a' as Varchar)", "PARSE_CHAR_MISSING_LENGTH", 1, 19, 19, - "DataType varchar requires a length parameter") + "DataType \"VARCHAR\" requires a length parameter") intercept("SELECT cast('a' as Character)", "PARSE_CHAR_MISSING_LENGTH", 1, 19, 19, - "DataType character requires a length parameter") + "DataType \"CHARACTER\" requires a length parameter") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index d9fc877c321cd..10b763b1b5134 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -497,8 +497,8 @@ object QueryExecution { case e @ (_: java.lang.NullPointerException | _: java.lang.AssertionError) => new SparkException( errorClass = "INTERNAL_ERROR", - messageParameters = Array(msg + - " Please, fill a bug report in, and provide the full stack trace."), + messageParameters = Map("message" -> (msg + + " Please, fill a bug report in, and provide the full stack trace.")), cause = e) case e: Throwable => e diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEPropagateEmptyRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEPropagateEmptyRelation.scala index 132c919c29112..7951a6f36b9bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEPropagateEmptyRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEPropagateEmptyRelation.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.catalyst.planning.ExtractSingleColumnNullAwareAntiJo import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.TreePattern.{LOCAL_RELATION, LOGICAL_QUERY_STAGE, TRUE_OR_FALSE_LITERAL} import org.apache.spark.sql.execution.aggregate.BaseAggregateExec +import org.apache.spark.sql.execution.exchange.{REPARTITION_BY_COL, REPARTITION_BY_NUM, ShuffleExchangeLike} import org.apache.spark.sql.execution.joins.HashedRelationWithAllNullKeys /** @@ -33,11 +34,16 @@ import org.apache.spark.sql.execution.joins.HashedRelationWithAllNullKeys */ object AQEPropagateEmptyRelation extends PropagateEmptyRelationBase { override protected def isEmpty(plan: LogicalPlan): Boolean = - super.isEmpty(plan) || getEstimatedRowCount(plan).contains(0) + super.isEmpty(plan) || (!isRootRepartition(plan) && getEstimatedRowCount(plan).contains(0)) override protected def nonEmpty(plan: LogicalPlan): Boolean = super.nonEmpty(plan) || getEstimatedRowCount(plan).exists(_ > 0) + private def isRootRepartition(plan: LogicalPlan): Boolean = plan match { + case l: LogicalQueryStage if l.getTagValue(ROOT_REPARTITION).isDefined => true + case _ => false + } + // The returned value follows: // - 0 means the plan must produce 0 row // - positive value means an estimated row count which can be over-estimated @@ -69,6 +75,13 @@ object AQEPropagateEmptyRelation extends PropagateEmptyRelationBase { empty(j) } + override protected def userSpecifiedRepartition(p: LogicalPlan): Boolean = p match { + case LogicalQueryStage(_, ShuffleQueryStageExec(_, shuffle: ShuffleExchangeLike, _)) + if shuffle.shuffleOrigin == REPARTITION_BY_COL || + shuffle.shuffleOrigin == REPARTITION_BY_NUM => true + case _ => false + } + override protected def applyInternal(p: LogicalPlan): LogicalPlan = p.transformUpWithPruning( // LOCAL_RELATION and TRUE_OR_FALSE_LITERAL pattern are matched at // `PropagateEmptyRelationBase.commonApplyFunc` diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEUtils.scala index 51833012a128e..1a0836ed752bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEUtils.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.adaptive import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution, HashPartitioning, UnspecifiedDistribution} -import org.apache.spark.sql.execution.{CollectMetricsExec, FilterExec, ProjectExec, SortExec, SparkPlan} +import org.apache.spark.sql.execution.{CollectMetricsExec, DeserializeToObjectExec, FilterExec, ProjectExec, SortExec, SparkPlan} import org.apache.spark.sql.execution.exchange.{REPARTITION_BY_COL, REPARTITION_BY_NUM, ShuffleExchangeExec} object AQEUtils { @@ -41,6 +41,7 @@ object AQEUtils { case f: FilterExec => getRequiredDistribution(f.child) case s: SortExec if !s.global => getRequiredDistribution(s.child) case c: CollectMetricsExec => getRequiredDistribution(c.child) + case d: DeserializeToObjectExec => getRequiredDistribution(d.child) case p: ProjectExec => getRequiredDistribution(p.child).flatMap { case h: ClusteredDistribution => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 44a5ba4a547d9..36d74fdd24d07 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -118,6 +118,7 @@ case class AdaptiveSparkPlanExec( Seq( RemoveRedundantProjects, ensureRequirements, + AdjustShuffleExchangePosition, ValidateSparkPlan, ReplaceHashWithSortAgg, RemoveRedundantSorts, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdjustShuffleExchangePosition.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdjustShuffleExchangePosition.scala new file mode 100644 index 0000000000000..f211b6cc8a069 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdjustShuffleExchangePosition.scala @@ -0,0 +1,43 @@ +/* + * 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.execution.adaptive + +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{DeserializeToObjectExec, SparkPlan} +import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike + +/** + * This rule is used to adjust the shuffle exchange with special SparkPlan who + * does not allow a shuffle on top of it. + */ +object AdjustShuffleExchangePosition extends Rule[SparkPlan] { + private def shouldAdjust(plan: SparkPlan): Boolean = plan match { + // `DeserializeToObjectExec` is used by Spark internally e.g. `Dataset.rdd`. It produces + // safe rows and must be root node because SQL operators only accept unsafe rows as input. + // This conflicts with AQE framework since we may add shuffle back during re-optimize + // to preserve the user-specified repartition, so here we adjust the position with shuffle. + case _: DeserializeToObjectExec => true + case _ => false + } + + override def apply(plan: SparkPlan): SparkPlan = plan match { + case shuffle: ShuffleExchangeLike if shouldAdjust(shuffle.child) => + shuffle.child.withNewChildren(shuffle.withNewChildren(shuffle.child.children) :: Nil) + case _ => plan + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStage.scala index f8b786778a798..5e6f1b5a88408 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStage.scala @@ -18,8 +18,8 @@ package org.apache.spark.sql.execution.adaptive import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} -import org.apache.spark.sql.catalyst.trees.TreePattern.{LOGICAL_QUERY_STAGE, TreePattern} +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, RepartitionOperation, Statistics} +import org.apache.spark.sql.catalyst.trees.TreePattern.{LOGICAL_QUERY_STAGE, REPARTITION_OPERATION, TreePattern} import org.apache.spark.sql.execution.SparkPlan /** @@ -40,7 +40,15 @@ case class LogicalQueryStage( override def output: Seq[Attribute] = logicalPlan.output override val isStreaming: Boolean = logicalPlan.isStreaming override val outputOrdering: Seq[SortOrder] = physicalPlan.outputOrdering - override protected val nodePatterns: Seq[TreePattern] = Seq(LOGICAL_QUERY_STAGE) + override protected val nodePatterns: Seq[TreePattern] = { + // Repartition is a special node that it represents a shuffle exchange, + // then in AQE the repartition will be always wrapped into `LogicalQueryStage` + val repartitionPattern = logicalPlan match { + case _: RepartitionOperation => Some(REPARTITION_OPERATION) + case _ => None + } + Seq(LOGICAL_QUERY_STAGE) ++ repartitionPattern + } override def computeStats(): Statistics = { // TODO this is not accurate when there is other physical nodes above QueryStageExec. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala index 137e2fe93c790..8467feb91d144 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala @@ -34,8 +34,8 @@ class ArrowPythonRunner( protected override val timeZoneId: String, protected override val workerConf: Map[String, String]) extends BasePythonRunner[Iterator[InternalRow], ColumnarBatch](funcs, evalType, argOffsets) - with PythonArrowInput - with PythonArrowOutput { + with BasicPythonArrowInput + with BasicPythonArrowOutput { override val simplifiedTraceback: Boolean = SQLConf.get.pysparkSimplifiedTraceback diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala index e3d8a943d8cf2..2661896ececc9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala @@ -48,7 +48,7 @@ class CoGroupedArrowPythonRunner( conf: Map[String, String]) extends BasePythonRunner[ (Iterator[InternalRow], Iterator[InternalRow]), ColumnarBatch](funcs, evalType, argOffsets) - with PythonArrowOutput { + with BasicPythonArrowOutput { override val simplifiedTraceback: Boolean = SQLConf.get.pysparkSimplifiedTraceback diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowInput.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowInput.scala index 79365080f8cb3..6168d0f867adb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowInput.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowInput.scala @@ -32,15 +32,21 @@ import org.apache.spark.util.Utils /** * A trait that can be mixed-in with [[BasePythonRunner]]. It implements the logic from - * JVM (an iterator of internal rows) to Python (Arrow). + * JVM (an iterator of internal rows + additional data if required) to Python (Arrow). */ -private[python] trait PythonArrowInput { self: BasePythonRunner[Iterator[InternalRow], _] => +private[python] trait PythonArrowInput[IN] { self: BasePythonRunner[IN, _] => protected val workerConf: Map[String, String] protected val schema: StructType protected val timeZoneId: String + protected def writeIteratorToArrowStream( + root: VectorSchemaRoot, + writer: ArrowStreamWriter, + dataOut: DataOutputStream, + inputIterator: Iterator[IN]): Unit + protected def handleMetadataBeforeExec(stream: DataOutputStream): Unit = { // Write config for the worker as a number of key -> value pairs of strings stream.writeInt(workerConf.size) @@ -53,7 +59,7 @@ private[python] trait PythonArrowInput { self: BasePythonRunner[Iterator[Interna protected override def newWriterThread( env: SparkEnv, worker: Socket, - inputIterator: Iterator[Iterator[InternalRow]], + inputIterator: Iterator[IN], partitionIndex: Int, context: TaskContext): WriterThread = { new WriterThread(env, worker, inputIterator, partitionIndex, context) { @@ -74,17 +80,8 @@ private[python] trait PythonArrowInput { self: BasePythonRunner[Iterator[Interna val writer = new ArrowStreamWriter(root, null, dataOut) writer.start() - while (inputIterator.hasNext) { - val nextBatch = inputIterator.next() - - while (nextBatch.hasNext) { - arrowWriter.write(nextBatch.next()) - } + writeIteratorToArrowStream(root, writer, dataOut, inputIterator) - arrowWriter.finish() - writer.writeBatch() - arrowWriter.reset() - } // end writes footer to the output stream and doesn't clean any resources. // It could throw exception if the output stream is closed, so it should be // in the try block. @@ -107,3 +104,27 @@ private[python] trait PythonArrowInput { self: BasePythonRunner[Iterator[Interna } } } + +private[python] trait BasicPythonArrowInput extends PythonArrowInput[Iterator[InternalRow]] { + self: BasePythonRunner[Iterator[InternalRow], _] => + + protected def writeIteratorToArrowStream( + root: VectorSchemaRoot, + writer: ArrowStreamWriter, + dataOut: DataOutputStream, + inputIterator: Iterator[Iterator[InternalRow]]): Unit = { + val arrowWriter = ArrowWriter.create(root) + + while (inputIterator.hasNext) { + val nextBatch = inputIterator.next() + + while (nextBatch.hasNext) { + arrowWriter.write(nextBatch.next()) + } + + arrowWriter.finish() + writer.writeBatch() + arrowWriter.reset() + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowOutput.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowOutput.scala index d06a0d012990b..339f114539c28 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowOutput.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowOutput.scala @@ -33,12 +33,14 @@ import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch, Column /** * A trait that can be mixed-in with [[BasePythonRunner]]. It implements the logic from - * Python (Arrow) to JVM (ColumnarBatch). + * Python (Arrow) to JVM (output type being deserialized from ColumnarBatch). */ -private[python] trait PythonArrowOutput { self: BasePythonRunner[_, ColumnarBatch] => +private[python] trait PythonArrowOutput[OUT <: AnyRef] { self: BasePythonRunner[_, OUT] => protected def handleMetadataAfterExec(stream: DataInputStream): Unit = { } + protected def deserializeColumnarBatch(batch: ColumnarBatch, schema: StructType): OUT + protected def newReaderIterator( stream: DataInputStream, writerThread: WriterThread, @@ -47,7 +49,7 @@ private[python] trait PythonArrowOutput { self: BasePythonRunner[_, ColumnarBatc worker: Socket, pid: Option[Int], releasedOrClosed: AtomicBoolean, - context: TaskContext): Iterator[ColumnarBatch] = { + context: TaskContext): Iterator[OUT] = { new ReaderIterator( stream, writerThread, startTime, env, worker, pid, releasedOrClosed, context) { @@ -74,7 +76,7 @@ private[python] trait PythonArrowOutput { self: BasePythonRunner[_, ColumnarBatc super.handleEndOfDataSection() } - protected override def read(): ColumnarBatch = { + protected override def read(): OUT = { if (writerThread.exception.isDefined) { throw writerThread.exception.get } @@ -84,7 +86,7 @@ private[python] trait PythonArrowOutput { self: BasePythonRunner[_, ColumnarBatc if (batchLoaded) { val batch = new ColumnarBatch(vectors) batch.setNumRows(root.getRowCount) - batch + deserializeColumnarBatch(batch, schema) } else { reader.close(false) allocator.close() @@ -108,7 +110,7 @@ private[python] trait PythonArrowOutput { self: BasePythonRunner[_, ColumnarBatc throw handlePythonException() case SpecialLengths.END_OF_DATA_SECTION => handleEndOfDataSection() - null + null.asInstanceOf[OUT] } } } catch handleException @@ -116,3 +118,11 @@ private[python] trait PythonArrowOutput { self: BasePythonRunner[_, ColumnarBatc } } } + +private[python] trait BasicPythonArrowOutput extends PythonArrowOutput[ColumnarBatch] { + self: BasePythonRunner[_, ColumnarBatch] => + + protected def deserializeColumnarBatch( + batch: ColumnarBatch, + schema: StructType): ColumnarBatch = batch +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 0640eb64a967c..7f183d744bc6e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -70,7 +70,7 @@ case class ScalarSubquery( override def withNewPlan(query: BaseSubqueryExec): ScalarSubquery = copy(plan = query) def initQueryContext(): Option[SQLQueryContext] = Some(origin.context) - override lazy val preCanonicalized: Expression = { + override lazy val canonicalized: Expression = { ScalarSubquery(plan.canonicalized.asInstanceOf[BaseSubqueryExec], ExprId(0)) } @@ -157,9 +157,9 @@ case class InSubqueryExec( inSet.doGenCode(ctx, ev) } - override lazy val preCanonicalized: InSubqueryExec = { + override lazy val canonicalized: InSubqueryExec = { copy( - child = child.preCanonicalized, + child = child.canonicalized, plan = plan.canonicalized.asInstanceOf[BaseSubqueryExec], exprId = ExprId(0), resultBroadcast = null, 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 b0b220bba5960..f11f987423660 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 @@ -168,9 +168,9 @@ org.apache.spark.SparkArrayIndexOutOfBoundsException { "errorClass" : "INVALID_ARRAY_INDEX_IN_ELEMENT_AT", "messageParameters" : { - "indexValue" : "5", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "arraySize" : "3", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "indexValue" : "5" }, "queryContext" : [ { "objectType" : "", @@ -191,9 +191,9 @@ org.apache.spark.SparkArrayIndexOutOfBoundsException { "errorClass" : "INVALID_ARRAY_INDEX_IN_ELEMENT_AT", "messageParameters" : { - "indexValue" : "-5", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "arraySize" : "3", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "indexValue" : "-5" }, "queryContext" : [ { "objectType" : "", @@ -232,9 +232,9 @@ org.apache.spark.SparkArrayIndexOutOfBoundsException { "errorClass" : "INVALID_ARRAY_INDEX", "messageParameters" : { - "indexValue" : "4", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "arraySize" : "2", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "indexValue" : "4" }, "queryContext" : [ { "objectType" : "", @@ -255,9 +255,9 @@ org.apache.spark.SparkArrayIndexOutOfBoundsException { "errorClass" : "INVALID_ARRAY_INDEX", "messageParameters" : { - "indexValue" : "0", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "arraySize" : "2", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "indexValue" : "0" }, "queryContext" : [ { "objectType" : "", @@ -278,9 +278,9 @@ org.apache.spark.SparkArrayIndexOutOfBoundsException { "errorClass" : "INVALID_ARRAY_INDEX", "messageParameters" : { - "indexValue" : "-1", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "arraySize" : "2", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "indexValue" : "-1" }, "queryContext" : [ { "objectType" : "", @@ -333,9 +333,9 @@ org.apache.spark.SparkArrayIndexOutOfBoundsException { "errorClass" : "INVALID_ARRAY_INDEX", "messageParameters" : { - "indexValue" : "5", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "arraySize" : "3", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "indexValue" : "5" }, "queryContext" : [ { "objectType" : "", @@ -356,9 +356,9 @@ org.apache.spark.SparkArrayIndexOutOfBoundsException { "errorClass" : "INVALID_ARRAY_INDEX", "messageParameters" : { - "indexValue" : "-1", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "arraySize" : "3", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "indexValue" : "-1" }, "queryContext" : [ { "objectType" : "", 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 378f817f7d9a6..deaece6e7e1ac 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 @@ -9,10 +9,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'1.23'", "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"INT\"" }, "queryContext" : [ { "objectType" : "", @@ -34,10 +34,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'1.23'", "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BIGINT\"" }, "queryContext" : [ { "objectType" : "", @@ -59,10 +59,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'-4.56'", "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"INT\"" }, "queryContext" : [ { "objectType" : "", @@ -84,10 +84,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'-4.56'", "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BIGINT\"" }, "queryContext" : [ { "objectType" : "", @@ -109,10 +109,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'abc'", "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"INT\"" }, "queryContext" : [ { "objectType" : "", @@ -134,10 +134,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'abc'", "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BIGINT\"" }, "queryContext" : [ { "objectType" : "", @@ -159,10 +159,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'abc'", "sourceType" : "\"STRING\"", - "targetType" : "\"FLOAT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"FLOAT\"" }, "queryContext" : [ { "objectType" : "", @@ -184,10 +184,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'abc'", "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"DOUBLE\"" }, "queryContext" : [ { "objectType" : "", @@ -209,10 +209,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'1234567890123'", "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"INT\"" }, "queryContext" : [ { "objectType" : "", @@ -234,10 +234,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'12345678901234567890123'", "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BIGINT\"" }, "queryContext" : [ { "objectType" : "", @@ -259,10 +259,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "''", "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"INT\"" }, "queryContext" : [ { "objectType" : "", @@ -284,10 +284,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "''", "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BIGINT\"" }, "queryContext" : [ { "objectType" : "", @@ -309,10 +309,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "''", "sourceType" : "\"STRING\"", - "targetType" : "\"FLOAT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"FLOAT\"" }, "queryContext" : [ { "objectType" : "", @@ -334,10 +334,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "''", "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"DOUBLE\"" }, "queryContext" : [ { "objectType" : "", @@ -375,10 +375,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'123.a'", "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"INT\"" }, "queryContext" : [ { "objectType" : "", @@ -400,10 +400,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'123.a'", "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BIGINT\"" }, "queryContext" : [ { "objectType" : "", @@ -425,10 +425,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'123.a'", "sourceType" : "\"STRING\"", - "targetType" : "\"FLOAT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"FLOAT\"" }, "queryContext" : [ { "objectType" : "", @@ -450,10 +450,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'123.a'", "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"DOUBLE\"" }, "queryContext" : [ { "objectType" : "", @@ -483,10 +483,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'-2147483649'", "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"INT\"" }, "queryContext" : [ { "objectType" : "", @@ -516,10 +516,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'2147483648'", "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"INT\"" }, "queryContext" : [ { "objectType" : "", @@ -549,10 +549,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'-9223372036854775809'", "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BIGINT\"" }, "queryContext" : [ { "objectType" : "", @@ -582,10 +582,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'9223372036854775808'", "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BIGINT\"" }, "queryContext" : [ { "objectType" : "", @@ -854,10 +854,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'1中文'", "sourceType" : "\"STRING\"", - "targetType" : "\"TINYINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"TINYINT\"" }, "queryContext" : [ { "objectType" : "", @@ -879,10 +879,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'1中文'", "sourceType" : "\"STRING\"", - "targetType" : "\"SMALLINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"SMALLINT\"" }, "queryContext" : [ { "objectType" : "", @@ -904,10 +904,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'1中文'", "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"INT\"" }, "queryContext" : [ { "objectType" : "", @@ -929,10 +929,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'中文1'", "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BIGINT\"" }, "queryContext" : [ { "objectType" : "", @@ -954,10 +954,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'1中文'", "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BIGINT\"" }, "queryContext" : [ { "objectType" : "", @@ -997,10 +997,10 @@ org.apache.spark.SparkRuntimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'\t\n xyz \t\r'", "sourceType" : "\"STRING\"", - "targetType" : "\"BOOLEAN\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BOOLEAN\"" }, "queryContext" : [ { "objectType" : "", @@ -1030,10 +1030,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE", "sqlState" : "22005", "messageParameters" : { - "value" : "123.45", + "config" : "\"spark.sql.ansi.enabled\"", "precision" : "4", "scale" : "2", - "config" : "\"spark.sql.ansi.enabled\"" + "value" : "123.45" }, "queryContext" : [ { "objectType" : "", @@ -1055,10 +1055,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'xyz'", "sourceType" : "\"STRING\"", - "targetType" : "\"DECIMAL(4,2)\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"DECIMAL(4,2)\"" }, "queryContext" : [ { "objectType" : "", @@ -1088,10 +1088,10 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'a'", "sourceType" : "\"STRING\"", - "targetType" : "\"DATE\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"DATE\"" }, "queryContext" : [ { "objectType" : "", @@ -1121,10 +1121,10 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'a'", "sourceType" : "\"STRING\"", - "targetType" : "\"TIMESTAMP\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"TIMESTAMP\"" }, "queryContext" : [ { "objectType" : "", @@ -1154,10 +1154,10 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'a'", "sourceType" : "\"STRING\"", - "targetType" : "\"TIMESTAMP_NTZ\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"TIMESTAMP_NTZ\"" }, "queryContext" : [ { "objectType" : "", @@ -1179,10 +1179,10 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "Infinity", "sourceType" : "\"DOUBLE\"", - "targetType" : "\"TIMESTAMP\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"TIMESTAMP\"" }, "queryContext" : [ { "objectType" : "", @@ -1204,10 +1204,10 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "Infinity", "sourceType" : "\"DOUBLE\"", - "targetType" : "\"TIMESTAMP\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"TIMESTAMP\"" }, "queryContext" : [ { "objectType" : "", @@ -1261,10 +1261,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "CAST_OVERFLOW", "sqlState" : "22005", "messageParameters" : { - "value" : "INTERVAL '23:59:59' HOUR TO SECOND", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "sourceType" : "\"INTERVAL HOUR TO SECOND\"", "targetType" : "\"SMALLINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "value" : "INTERVAL '23:59:59' HOUR TO SECOND" } } @@ -1295,10 +1295,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "CAST_OVERFLOW", "sqlState" : "22005", "messageParameters" : { - "value" : "INTERVAL '-1000' MONTH", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "sourceType" : "\"INTERVAL MONTH\"", "targetType" : "\"TINYINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "value" : "INTERVAL '-1000' MONTH" } } @@ -1313,10 +1313,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "CAST_OVERFLOW", "sqlState" : "22005", "messageParameters" : { - "value" : "INTERVAL '1000000' SECOND", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "sourceType" : "\"INTERVAL SECOND\"", "targetType" : "\"SMALLINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "value" : "INTERVAL '1000000' SECOND" } } @@ -1403,10 +1403,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "CAST_OVERFLOW", "sqlState" : "22005", "messageParameters" : { - "value" : "2147483647", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "sourceType" : "\"INT\"", "targetType" : "\"INTERVAL YEAR\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "value" : "2147483647" } } @@ -1421,10 +1421,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "CAST_OVERFLOW", "sqlState" : "22005", "messageParameters" : { - "value" : "-9223372036854775808L", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "sourceType" : "\"BIGINT\"", "targetType" : "\"INTERVAL DAY\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "value" : "-9223372036854775808L" } } @@ -1495,10 +1495,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE", "sqlState" : "22005", "messageParameters" : { - "value" : "10.123000", + "config" : "\"spark.sql.ansi.enabled\"", "precision" : "1", "scale" : "0", - "config" : "\"spark.sql.ansi.enabled\"" + "value" : "10.123000" }, "queryContext" : [ { "objectType" : "", 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 83ff07bc43fac..73b9768b5fcb2 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 @@ -141,8 +141,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Invalid date 'February 29' as '1970' is not a leap year", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Invalid date 'February 29' as '1970' is not a leap year" } } @@ -240,10 +240,10 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'xx'", "sourceType" : "\"STRING\"", - "targetType" : "\"DATE\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"DATE\"" }, "queryContext" : [ { "objectType" : "", @@ -348,10 +348,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'1.2'", "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"INT\"" }, "queryContext" : [ { "objectType" : "", @@ -472,10 +472,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'1.2'", "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"INT\"" }, "queryContext" : [ { "objectType" : "", @@ -697,8 +697,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd/MMMMM/yyyy'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd/MMMMM/yyyy'" } } @@ -713,8 +713,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd/MMMMM/yyyy'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd/MMMMM/yyyy'" } } @@ -729,8 +729,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd/MMMMM/yyyy'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd/MMMMM/yyyy'" } } diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out index 507ce620b8995..464f055b35ca0 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out @@ -18,8 +18,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "PARSE_DATETIME_BY_NEW_PARSER", "messageParameters" : { - "datetime" : "'1'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "datetime" : "'1'" } } @@ -34,8 +34,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text '-12' could not be parsed at index 0", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '-12' could not be parsed at index 0" } } @@ -50,8 +50,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "PARSE_DATETIME_BY_NEW_PARSER", "messageParameters" : { - "datetime" : "'123'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "datetime" : "'123'" } } @@ -66,8 +66,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "PARSE_DATETIME_BY_NEW_PARSER", "messageParameters" : { - "datetime" : "'1'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "datetime" : "'1'" } } @@ -82,8 +82,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'yyyyyyy'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'yyyyyyy'" } } @@ -98,8 +98,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Invalid date 'DayOfYear 366' as '1970' is not a leap year", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Invalid date 'DayOfYear 366' as '1970' is not a leap year" } } @@ -114,8 +114,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "PARSE_DATETIME_BY_NEW_PARSER", "messageParameters" : { - "datetime" : "'9'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "datetime" : "'9'" } } @@ -130,8 +130,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "PARSE_DATETIME_BY_NEW_PARSER", "messageParameters" : { - "datetime" : "'9'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "datetime" : "'9'" } } @@ -146,8 +146,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "PARSE_DATETIME_BY_NEW_PARSER", "messageParameters" : { - "datetime" : "'99'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "datetime" : "'99'" } } @@ -162,8 +162,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31" } } @@ -178,8 +178,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Conflict found: Field MonthOfYear 11 differs from MonthOfYear 12 derived from 1970-12-31", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Conflict found: Field MonthOfYear 11 differs from MonthOfYear 12 derived from 1970-12-31" } } @@ -194,8 +194,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text '2019-366' could not be parsed: Invalid date 'DayOfYear 366' as '2019' is not a leap year", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2019-366' could not be parsed: Invalid date 'DayOfYear 366' as '2019' is not a leap year" } } @@ -210,8 +210,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31" } } @@ -226,8 +226,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text '2020-01-365' could not be parsed: Conflict found: Field DayOfMonth 30 differs from DayOfMonth 1 derived from 2020-12-30", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2020-01-365' could not be parsed: Conflict found: Field DayOfMonth 30 differs from DayOfMonth 1 derived from 2020-12-30" } } @@ -242,8 +242,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text '2020-10-350' could not be parsed: Conflict found: Field MonthOfYear 12 differs from MonthOfYear 10 derived from 2020-12-15", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2020-10-350' could not be parsed: Conflict found: Field MonthOfYear 12 differs from MonthOfYear 10 derived from 2020-12-15" } } @@ -258,8 +258,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text '2020-11-31-366' could not be parsed: Invalid date 'NOVEMBER 31'", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2020-11-31-366' could not be parsed: Invalid date 'NOVEMBER 31'" } } @@ -274,8 +274,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "PARSE_DATETIME_BY_NEW_PARSER", "messageParameters" : { - "datetime" : "'2018-366'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "datetime" : "'2018-366'" } } @@ -290,8 +290,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" } } @@ -306,8 +306,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text 'Unparseable' could not be parsed at index 0", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text 'Unparseable' could not be parsed at index 0" } } @@ -322,8 +322,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" } } @@ -338,8 +338,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text 'Unparseable' could not be parsed at index 0", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text 'Unparseable' could not be parsed at index 0" } } @@ -354,8 +354,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" } } @@ -370,8 +370,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text 'Unparseable' could not be parsed at index 0", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text 'Unparseable' could not be parsed at index 0" } } @@ -386,8 +386,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" } } @@ -402,8 +402,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text 'Unparseable' could not be parsed at index 0", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text 'Unparseable' could not be parsed at index 0" } } @@ -418,10 +418,10 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'Unparseable'", "sourceType" : "\"STRING\"", - "targetType" : "\"TIMESTAMP\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"TIMESTAMP\"" }, "queryContext" : [ { "objectType" : "", @@ -443,10 +443,10 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'Unparseable'", "sourceType" : "\"STRING\"", - "targetType" : "\"DATE\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"DATE\"" }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out index ff799d199d92a..a0a40e06432b2 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out @@ -77,10 +77,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE", "sqlState" : "22005", "messageParameters" : { - "value" : "10000000000000000000000000000000000000.1", + "config" : "\"spark.sql.ansi.enabled\"", "precision" : "38", "scale" : "1", - "config" : "\"spark.sql.ansi.enabled\"" + "value" : "10000000000000000000000000000000000000.1" }, "queryContext" : [ { "objectType" : "", @@ -102,10 +102,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE", "sqlState" : "22005", "messageParameters" : { - "value" : "-11000000000000000000000000000000000000.1", + "config" : "\"spark.sql.ansi.enabled\"", "precision" : "38", "scale" : "1", - "config" : "\"spark.sql.ansi.enabled\"" + "value" : "-11000000000000000000000000000000000000.1" }, "queryContext" : [ { "objectType" : "", @@ -127,10 +127,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE", "sqlState" : "22005", "messageParameters" : { - "value" : "152415787532388367501905199875019052100", + "config" : "\"spark.sql.ansi.enabled\"", "precision" : "38", "scale" : "2", - "config" : "\"spark.sql.ansi.enabled\"" + "value" : "152415787532388367501905199875019052100" }, "queryContext" : [ { "objectType" : "", @@ -152,10 +152,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE", "sqlState" : "22005", "messageParameters" : { - "value" : "1000000000000000000000000000000000000.00000000000000000000000000000000000000", + "config" : "\"spark.sql.ansi.enabled\"", "precision" : "38", "scale" : "6", - "config" : "\"spark.sql.ansi.enabled\"" + "value" : "1000000000000000000000000000000000000.00000000000000000000000000000000000000" }, "queryContext" : [ { "objectType" : "", @@ -201,10 +201,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE", "sqlState" : "22005", "messageParameters" : { - "value" : "10123456789012345678901234567890123456.00000000000000000000000000000000000000", + "config" : "\"spark.sql.ansi.enabled\"", "precision" : "38", "scale" : "6", - "config" : "\"spark.sql.ansi.enabled\"" + "value" : "10123456789012345678901234567890123456.00000000000000000000000000000000000000" }, "queryContext" : [ { "objectType" : "", @@ -226,10 +226,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE", "sqlState" : "22005", "messageParameters" : { - "value" : "101234567890123456789012345678901234.56000000000000000000000000000000000000", + "config" : "\"spark.sql.ansi.enabled\"", "precision" : "38", "scale" : "6", - "config" : "\"spark.sql.ansi.enabled\"" + "value" : "101234567890123456789012345678901234.56000000000000000000000000000000000000" }, "queryContext" : [ { "objectType" : "", @@ -251,10 +251,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE", "sqlState" : "22005", "messageParameters" : { - "value" : "10123456789012345678901234567890123.45600000000000000000000000000000000000", + "config" : "\"spark.sql.ansi.enabled\"", "precision" : "38", "scale" : "6", - "config" : "\"spark.sql.ansi.enabled\"" + "value" : "10123456789012345678901234567890123.45600000000000000000000000000000000000" }, "queryContext" : [ { "objectType" : "", @@ -276,10 +276,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE", "sqlState" : "22005", "messageParameters" : { - "value" : "1012345678901234567890123456789012.34560000000000000000000000000000000000", + "config" : "\"spark.sql.ansi.enabled\"", "precision" : "38", "scale" : "6", - "config" : "\"spark.sql.ansi.enabled\"" + "value" : "1012345678901234567890123456789012.34560000000000000000000000000000000000" }, "queryContext" : [ { "objectType" : "", @@ -301,10 +301,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE", "sqlState" : "22005", "messageParameters" : { - "value" : "101234567890123456789012345678901.23456000000000000000000000000000000000", + "config" : "\"spark.sql.ansi.enabled\"", "precision" : "38", "scale" : "6", - "config" : "\"spark.sql.ansi.enabled\"" + "value" : "101234567890123456789012345678901.23456000000000000000000000000000000000" }, "queryContext" : [ { "objectType" : "", @@ -334,10 +334,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE", "sqlState" : "22005", "messageParameters" : { - "value" : "101234567890123456789012345678901.23456000000000000000000000000000000000", + "config" : "\"spark.sql.ansi.enabled\"", "precision" : "38", "scale" : "6", - "config" : "\"spark.sql.ansi.enabled\"" + "value" : "101234567890123456789012345678901.23456000000000000000000000000000000000" }, "queryContext" : [ { "objectType" : "", 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 9cc649878044f..4c1e840f8f385 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 @@ -123,10 +123,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'a'", "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"DOUBLE\"" }, "queryContext" : [ { "objectType" : "", @@ -148,10 +148,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'a'", "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"DOUBLE\"" }, "queryContext" : [ { "objectType" : "", @@ -173,10 +173,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'a'", "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"DOUBLE\"" }, "queryContext" : [ { "objectType" : "", @@ -198,10 +198,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'a'", "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"DOUBLE\"" }, "queryContext" : [ { "objectType" : "", @@ -239,10 +239,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'a'", "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"DOUBLE\"" }, "queryContext" : [ { "objectType" : "", @@ -264,10 +264,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'a'", "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"DOUBLE\"" }, "queryContext" : [ { "objectType" : "", @@ -289,16 +289,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(2 / INTERVAL '02' SECOND)\"", "left" : "\"STRING\"", - "right" : "\"INTERVAL SECOND\"" + "right" : "\"INTERVAL SECOND\"", + "sqlExpr" : "\"(2 / INTERVAL '02' SECOND)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 30, - "fragment" : "select '2' / interval 2 second" + "fragment" : "'2' / interval 2 second" } ] } @@ -313,16 +313,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"", "left" : "\"STRING\"", - "right" : "\"INTERVAL YEAR\"" + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 28, - "fragment" : "select '2' / interval 2 year" + "fragment" : "'2' / interval 2 year" } ] } @@ -423,16 +423,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"", "left" : "\"INT\"", - "right" : "\"INTERVAL YEAR\"" + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 28, - "fragment" : "select 2 / interval '2' year" + "fragment" : "2 / interval '2' year" } ] } @@ -447,16 +447,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(2 / INTERVAL '02' HOUR)\"", "left" : "\"INT\"", - "right" : "\"INTERVAL HOUR\"" + "right" : "\"INTERVAL HOUR\"", + "sqlExpr" : "\"(2 / INTERVAL '02' HOUR)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 28, - "fragment" : "select 2 / interval '2' hour" + "fragment" : "2 / interval '2' hour" } ] } @@ -471,16 +471,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(NULL / INTERVAL '2' YEAR)\"", "left" : "\"VOID\"", - "right" : "\"INTERVAL YEAR\"" + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(NULL / INTERVAL '2' YEAR)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 31, - "fragment" : "select null / interval '2' year" + "fragment" : "null / interval '2' year" } ] } @@ -495,16 +495,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(NULL / INTERVAL '02' HOUR)\"", "left" : "\"VOID\"", - "right" : "\"INTERVAL HOUR\"" + "right" : "\"INTERVAL HOUR\"", + "sqlExpr" : "\"(NULL / INTERVAL '02' HOUR)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 31, - "fragment" : "select null / interval '2' hour" + "fragment" : "null / interval '2' hour" } ] } @@ -847,10 +847,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE", "sqlState" : "22005", "messageParameters" : { - "value" : "1234567890123456789", + "config" : "\"spark.sql.ansi.enabled\"", "precision" : "18", "scale" : "6", - "config" : "\"spark.sql.ansi.enabled\"" + "value" : "1234567890123456789" }, "queryContext" : [ { "objectType" : "", @@ -1673,16 +1673,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3 year to month)\"", "left" : "\"INTERVAL YEAR\"", - "right" : "\"STRING\"" + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3 year to month)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 388, - "fragment" : "select\n interval '2' year + '3-3 year to month',\n interval '2' year - '3 month',\n '3-2 year to month' + interval '2-2' year to month,\n '3 year' - interval '2-2' year to month,\n interval '99 11:22:33.123456789' day to second + '12:12 hour to second',\n interval '99 11:22:33.123456789' day to second - '12 hour',\n '4 day' + interval '10' day,\n '4 22 day to hour' - interval '10' day" + "startIndex" : 10, + "stopIndex" : 48, + "fragment" : "interval '2' year + '3-3 year to month'" } ] } @@ -1713,16 +1713,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3)\"", "left" : "\"INTERVAL YEAR\"", - "right" : "\"STRING\"" + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 32, - "fragment" : "select interval '2' year + '3-3'" + "fragment" : "interval '2' year + '3-3'" } ] } @@ -1737,16 +1737,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '2' YEAR - 4)\"", "left" : "\"INTERVAL YEAR\"", - "right" : "\"STRING\"" + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR - 4)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 30, - "fragment" : "select interval '2' year - '4'" + "fragment" : "interval '2' year - '4'" } ] } @@ -1761,10 +1761,10 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'4 11:11'", "sourceType" : "\"STRING\"", - "targetType" : "\"TIMESTAMP\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"TIMESTAMP\"" }, "queryContext" : [ { "objectType" : "", @@ -1786,10 +1786,10 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'4 12:12:12'", "sourceType" : "\"STRING\"", - "targetType" : "\"TIMESTAMP\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"TIMESTAMP\"" }, "queryContext" : [ { "objectType" : "", @@ -1819,16 +1819,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '2' YEAR + str)\"", "left" : "\"INTERVAL YEAR\"", - "right" : "\"STRING\"" + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + str)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 49, - "fragment" : "select interval '2' year + str from interval_view" + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' year + str" } ] } @@ -1843,16 +1843,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '2' YEAR - str)\"", "left" : "\"INTERVAL YEAR\"", - "right" : "\"STRING\"" + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR - str)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 49, - "fragment" : "select interval '2' year - str from interval_view" + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' year - str" } ] } @@ -1867,10 +1867,10 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'1'", "sourceType" : "\"STRING\"", - "targetType" : "\"TIMESTAMP\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"TIMESTAMP\"" }, "queryContext" : [ { "objectType" : "", @@ -1892,10 +1892,10 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'1'", "sourceType" : "\"STRING\"", - "targetType" : "\"TIMESTAMP\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"TIMESTAMP\"" }, "queryContext" : [ { "objectType" : "", @@ -1944,16 +1944,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '3' DAY - INTERVAL '2-2' YEAR TO MONTH)\"", "left" : "\"INTERVAL DAY\"", - "right" : "\"INTERVAL YEAR TO MONTH\"" + "right" : "\"INTERVAL YEAR TO MONTH\"", + "sqlExpr" : "\"(INTERVAL '3' DAY - INTERVAL '2-2' YEAR TO MONTH)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 54, - "fragment" : "select interval '3' day - interval '2-2' year to month" + "fragment" : "interval '3' day - interval '2-2' year to month" } ] } @@ -1977,16 +1977,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 + INTERVAL '2' MONTH)\"", "left" : "\"INT\"", - "right" : "\"INTERVAL MONTH\"" + "right" : "\"INTERVAL MONTH\"", + "sqlExpr" : "\"(1 + INTERVAL '2' MONTH)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 29, - "fragment" : "select 1 + interval '2' month" + "fragment" : "1 + interval '2' month" } ] } @@ -2010,16 +2010,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '2' MONTH - 1)\"", "left" : "\"INTERVAL MONTH\"", - "right" : "\"INT\"" + "right" : "\"INT\"", + "sqlExpr" : "\"(INTERVAL '2' MONTH - 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 29, - "fragment" : "select interval '2' month - 1" + "fragment" : "interval '2' month - 1" } ] } @@ -2161,8 +2161,8 @@ org.apache.spark.SparkArithmeticException "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "integer overflow", - "alternative" : "" + "alternative" : "", + "message" : "integer overflow" } } @@ -2177,8 +2177,8 @@ org.apache.spark.SparkArithmeticException "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "integer overflow", - "alternative" : " Use 'try_subtract' to tolerate overflow and return NULL instead." + "alternative" : " Use 'try_subtract' to tolerate overflow and return NULL instead.", + "message" : "integer overflow" } } @@ -2193,8 +2193,8 @@ org.apache.spark.SparkArithmeticException "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "integer overflow", - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead." + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "message" : "integer overflow" } } @@ -2429,8 +2429,8 @@ org.apache.spark.SparkArithmeticException "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "Interval value overflows after being divided by -1", - "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead." + "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.", + "message" : "Interval value overflows after being divided by -1" }, "queryContext" : [ { "objectType" : "", @@ -2452,8 +2452,8 @@ org.apache.spark.SparkArithmeticException "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "Interval value overflows after being divided by -1", - "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead." + "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.", + "message" : "Interval value overflows after being divided by -1" }, "queryContext" : [ { "objectType" : "", @@ -2509,8 +2509,8 @@ org.apache.spark.SparkArithmeticException "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "Interval value overflows after being divided by -1", - "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead." + "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.", + "message" : "Interval value overflows after being divided by -1" }, "queryContext" : [ { "objectType" : "", @@ -2532,8 +2532,8 @@ org.apache.spark.SparkArithmeticException "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "Interval value overflows after being divided by -1", - "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead." + "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.", + "message" : "Interval value overflows after being divided by -1" }, "queryContext" : [ { "objectType" : "", @@ -2881,16 +2881,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '1' MONTH > INTERVAL '20' DAY)\"", "left" : "\"INTERVAL MONTH\"", - "right" : "\"INTERVAL DAY\"" + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(INTERVAL '1' MONTH > INTERVAL '20' DAY)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 42, - "fragment" : "SELECT INTERVAL 1 MONTH > INTERVAL 20 DAYS" + "fragment" : "INTERVAL 1 MONTH > INTERVAL 20 DAYS" } ] } @@ -2905,16 +2905,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '1' DAY < 1)\"", "left" : "\"INTERVAL DAY\"", - "right" : "\"STRING\"" + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY < 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 29, - "fragment" : "SELECT INTERVAL '1' DAY < '1'" + "fragment" : "INTERVAL '1' DAY < '1'" } ] } @@ -2929,16 +2929,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '1' DAY = 1)\"", "left" : "\"INTERVAL DAY\"", - "right" : "\"STRING\"" + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY = 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 29, - "fragment" : "SELECT INTERVAL '1' DAY = '1'" + "fragment" : "INTERVAL '1' DAY = '1'" } ] } @@ -2953,16 +2953,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '1' DAY > 1)\"", "left" : "\"INTERVAL DAY\"", - "right" : "\"STRING\"" + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY > 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 29, - "fragment" : "SELECT INTERVAL '1' DAY > '1'" + "fragment" : "INTERVAL '1' DAY > '1'" } ] } @@ -2977,16 +2977,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 < INTERVAL '1' DAY)\"", "left" : "\"STRING\"", - "right" : "\"INTERVAL DAY\"" + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 < INTERVAL '1' DAY)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 29, - "fragment" : "SELECT '1' < INTERVAL '1' DAY" + "fragment" : "'1' < INTERVAL '1' DAY" } ] } @@ -3001,16 +3001,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 = INTERVAL '1' DAY)\"", "left" : "\"STRING\"", - "right" : "\"INTERVAL DAY\"" + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 = INTERVAL '1' DAY)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 29, - "fragment" : "SELECT '1' = INTERVAL '1' DAY" + "fragment" : "'1' = INTERVAL '1' DAY" } ] } @@ -3025,16 +3025,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 > INTERVAL '1' DAY)\"", "left" : "\"STRING\"", - "right" : "\"INTERVAL DAY\"" + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 > INTERVAL '1' DAY)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 29, - "fragment" : "SELECT '1' > INTERVAL '1' DAY" + "fragment" : "'1' > INTERVAL '1' DAY" } ] } @@ -3049,16 +3049,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '1' YEAR < 1)\"", "left" : "\"INTERVAL YEAR\"", - "right" : "\"STRING\"" + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR < 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 30, - "fragment" : "SELECT INTERVAL '1' YEAR < '1'" + "fragment" : "INTERVAL '1' YEAR < '1'" } ] } @@ -3073,16 +3073,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '1' YEAR = 1)\"", "left" : "\"INTERVAL YEAR\"", - "right" : "\"STRING\"" + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR = 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 30, - "fragment" : "SELECT INTERVAL '1' YEAR = '1'" + "fragment" : "INTERVAL '1' YEAR = '1'" } ] } @@ -3097,16 +3097,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '1' YEAR > 1)\"", "left" : "\"INTERVAL YEAR\"", - "right" : "\"STRING\"" + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR > 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 30, - "fragment" : "SELECT INTERVAL '1' YEAR > '1'" + "fragment" : "INTERVAL '1' YEAR > '1'" } ] } @@ -3121,16 +3121,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 < INTERVAL '1' YEAR)\"", "left" : "\"STRING\"", - "right" : "\"INTERVAL YEAR\"" + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 < INTERVAL '1' YEAR)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 30, - "fragment" : "SELECT '1' < INTERVAL '1' YEAR" + "fragment" : "'1' < INTERVAL '1' YEAR" } ] } @@ -3145,16 +3145,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 = INTERVAL '1' YEAR)\"", "left" : "\"STRING\"", - "right" : "\"INTERVAL YEAR\"" + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 = INTERVAL '1' YEAR)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 30, - "fragment" : "SELECT '1' = INTERVAL '1' YEAR" + "fragment" : "'1' = INTERVAL '1' YEAR" } ] } @@ -3169,16 +3169,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 > INTERVAL '1' YEAR)\"", "left" : "\"STRING\"", - "right" : "\"INTERVAL YEAR\"" + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 > INTERVAL '1' YEAR)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 30, - "fragment" : "SELECT '1' > INTERVAL '1' YEAR" + "fragment" : "'1' > INTERVAL '1' YEAR" } ] } @@ -3291,16 +3291,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '1' MONTH div INTERVAL '-1' DAY)\"", "left" : "\"INTERVAL MONTH\"", - "right" : "\"INTERVAL DAY\"" + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(INTERVAL '1' MONTH div INTERVAL '-1' DAY)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 49, - "fragment" : "SELECT div(INTERVAL '1' MONTH, INTERVAL '-1' DAY)" + "fragment" : "div(INTERVAL '1' MONTH, INTERVAL '-1' DAY)" } ] } 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 810f1942be233..4e04a29a98cf1 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 @@ -83,10 +83,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'a'", "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"INT\"" }, "queryContext" : [ { "objectType" : "", @@ -124,10 +124,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'a'", "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"INT\"" }, "queryContext" : [ { "objectType" : "", @@ -488,10 +488,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'invalid_length'", "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"INT\"" }, "queryContext" : [ { "objectType" : "", @@ -513,10 +513,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'invalid_length'", "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"INT\"" }, "queryContext" : [ { "objectType" : "", @@ -1187,10 +1187,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "' ab cdef= = '", "fmt" : "'BASE64'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "' ab cdef= = '", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1214,10 +1214,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "'a'", "fmt" : "'BASE64'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "'a'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1231,10 +1231,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "'a?'", "fmt" : "'BASE64'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "'a?'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1248,10 +1248,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "'abcde'", "fmt" : "'BASE64'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "'abcde'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1265,10 +1265,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "'abcd='", "fmt" : "'BASE64'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "'abcd='", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1282,10 +1282,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "'a==='", "fmt" : "'BASE64'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "'a==='", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1299,10 +1299,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "'ab==f'", "fmt" : "'BASE64'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "'ab==f'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1389,10 +1389,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "'GG'", "fmt" : "'HEX'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "'GG'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1406,10 +1406,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "'01 AF'", "fmt" : "'HEX'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "'01 AF'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1431,10 +1431,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "' ab cdef= = '", "fmt" : "'BASE64'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "' ab cdef= = '", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1448,10 +1448,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "' ab cdef= = '", "fmt" : "'HEX'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "' ab cdef= = '", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } 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 7969ccf1162d2..bd675df539e77 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 @@ -347,8 +347,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text '2019-10-06 10:11:12.' could not be parsed at index 20", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2019-10-06 10:11:12.' could not be parsed at index 20" } } @@ -419,8 +419,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text '2019-10-06 10:11:12.1234567PST' could not be parsed, unparsed text found at index 26", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2019-10-06 10:11:12.1234567PST' could not be parsed, unparsed text found at index 26" } } @@ -443,8 +443,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text '223456 2019-10-06 10:11:12.123456PST' could not be parsed at index 27", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '223456 2019-10-06 10:11:12.123456PST' could not be parsed at index 27" } } @@ -515,8 +515,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 7", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 7" } } @@ -531,8 +531,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 9", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 9" } } @@ -611,8 +611,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Invalid date 'February 29' as '1970' is not a leap year", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Invalid date 'February 29' as '1970' is not a leap year" } } @@ -723,16 +723,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_WRONG_TYPE", "messageParameters" : { - "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"", + "actualDataType" : "\"TIMESTAMP\"", "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", - "actualDataType" : "\"TIMESTAMP\"" + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 43, - "fragment" : "select timestamp'2011-11-11 11:11:11' + '1'" + "fragment" : "timestamp'2011-11-11 11:11:11' + '1'" } ] } @@ -747,16 +747,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_WRONG_TYPE", "messageParameters" : { - "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"", + "actualDataType" : "\"TIMESTAMP\"", "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", - "actualDataType" : "\"TIMESTAMP\"" + "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 43, - "fragment" : "select '1' + timestamp'2011-11-11 11:11:11'" + "fragment" : "'1' + timestamp'2011-11-11 11:11:11'" } ] } @@ -771,16 +771,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\"", "left" : "\"TIMESTAMP\"", - "right" : "\"VOID\"" + "right" : "\"VOID\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 44, - "fragment" : "select timestamp'2011-11-11 11:11:11' + null" + "fragment" : "timestamp'2011-11-11 11:11:11' + null" } ] } @@ -795,16 +795,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\"", "left" : "\"VOID\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 44, - "fragment" : "select null + timestamp'2011-11-11 11:11:11'" + "fragment" : "null + timestamp'2011-11-11 11:11:11'" } ] } @@ -842,8 +842,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'yyyy-MM-dd GGGGG'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'yyyy-MM-dd GGGGG'" } } @@ -858,8 +858,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd MM yyyy EEEEEE'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd MM yyyy EEEEEE'" } } @@ -874,8 +874,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd MM yyyy EEEEE'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd MM yyyy EEEEE'" } } @@ -890,8 +890,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd MM yyyy EEEEE'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd MM yyyy EEEEE'" } } @@ -906,8 +906,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd/MMMMM/yyyy'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd/MMMMM/yyyy'" } } @@ -922,8 +922,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd/MMMMM/yyyy'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd/MMMMM/yyyy'" } } diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/try_aggregates.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/try_aggregates.sql.out index fa55ace4d2e5f..94048ac8897bb 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/try_aggregates.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/try_aggregates.sql.out @@ -155,9 +155,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "long overflow", "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" }, "queryContext" : [ { "objectType" : "", @@ -363,9 +363,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "long overflow", "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" }, "queryContext" : [ { "objectType" : "", 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 914ee064c5127..9287ea356023f 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 @@ -49,9 +49,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "integer overflow", "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" }, "queryContext" : [ { "objectType" : "", @@ -73,9 +73,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "long overflow", "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" }, "queryContext" : [ { "objectType" : "", @@ -264,9 +264,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "integer overflow", "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" }, "queryContext" : [ { "objectType" : "", @@ -288,9 +288,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "long overflow", "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" }, "queryContext" : [ { "objectType" : "", @@ -422,9 +422,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "integer overflow", "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" }, "queryContext" : [ { "objectType" : "", @@ -446,9 +446,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "long overflow", "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" }, "queryContext" : [ { "objectType" : "", @@ -564,9 +564,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "integer overflow", "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" }, "queryContext" : [ { "objectType" : "", @@ -588,9 +588,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "long overflow", "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/try_datetime_functions.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/try_datetime_functions.sql.out index ab2172f56de86..c49637869e1fc 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/try_datetime_functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/try_datetime_functions.sql.out @@ -49,7 +49,7 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd MM yyyy EEEEEE'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd MM yyyy EEEEEE'" } } diff --git a/sql/core/src/test/resources/sql-tests/results/cast.sql.out b/sql/core/src/test/resources/sql-tests/results/cast.sql.out index 9271fd6350e47..0b9912160e448 100644 --- a/sql/core/src/test/resources/sql-tests/results/cast.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cast.sql.out @@ -635,10 +635,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "CAST_OVERFLOW", "sqlState" : "22005", "messageParameters" : { - "value" : "INTERVAL '23:59:59' HOUR TO SECOND", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "sourceType" : "\"INTERVAL HOUR TO SECOND\"", "targetType" : "\"SMALLINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "value" : "INTERVAL '23:59:59' HOUR TO SECOND" } } @@ -669,10 +669,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "CAST_OVERFLOW", "sqlState" : "22005", "messageParameters" : { - "value" : "INTERVAL '-1000' MONTH", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "sourceType" : "\"INTERVAL MONTH\"", "targetType" : "\"TINYINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "value" : "INTERVAL '-1000' MONTH" } } @@ -687,10 +687,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "CAST_OVERFLOW", "sqlState" : "22005", "messageParameters" : { - "value" : "INTERVAL '1000000' SECOND", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "sourceType" : "\"INTERVAL SECOND\"", "targetType" : "\"SMALLINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "value" : "INTERVAL '1000000' SECOND" } } @@ -777,10 +777,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "CAST_OVERFLOW", "sqlState" : "22005", "messageParameters" : { - "value" : "2147483647", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "sourceType" : "\"INT\"", "targetType" : "\"INTERVAL YEAR\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "value" : "2147483647" } } @@ -795,10 +795,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "CAST_OVERFLOW", "sqlState" : "22005", "messageParameters" : { - "value" : "-9223372036854775808L", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "sourceType" : "\"BIGINT\"", "targetType" : "\"INTERVAL DAY\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "value" : "-9223372036854775808L" } } @@ -869,10 +869,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE", "sqlState" : "22005", "messageParameters" : { - "value" : "10.123000", + "config" : "\"spark.sql.ansi.enabled\"", "precision" : "1", "scale" : "0", - "config" : "\"spark.sql.ansi.enabled\"" + "value" : "10.123000" } } diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out index b987fe7c02ad6..68e827ad55cc0 100644 --- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -52,8 +52,8 @@ org.apache.spark.sql.AnalysisException "errorSubClass" : "TABLE_OPERATION", "sqlState" : "0A000", "messageParameters" : { - "tableName" : "`spark_catalog`.`default`.`test_change`", - "operation" : "RENAME COLUMN" + "operation" : "RENAME COLUMN", + "tableName" : "`spark_catalog`.`default`.`test_change`" } } @@ -98,8 +98,8 @@ org.apache.spark.sql.AnalysisException "errorSubClass" : "TABLE_OPERATION", "sqlState" : "0A000", "messageParameters" : { - "tableName" : "`spark_catalog`.`default`.`test_change`", - "operation" : "ALTER COLUMN ... FIRST | ALTER" + "operation" : "ALTER COLUMN ... FIRST | ALTER", + "tableName" : "`spark_catalog`.`default`.`test_change`" } } @@ -115,8 +115,8 @@ org.apache.spark.sql.AnalysisException "errorSubClass" : "TABLE_OPERATION", "sqlState" : "0A000", "messageParameters" : { - "tableName" : "`spark_catalog`.`default`.`test_change`", - "operation" : "ALTER COLUMN ... FIRST | ALTER" + "operation" : "ALTER COLUMN ... FIRST | ALTER", + "tableName" : "`spark_catalog`.`default`.`test_change`" } } 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 3b25d183d5d65..253637eecf4ae 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 @@ -169,9 +169,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 34, - "fragment" : "SELECT db1.t1.i1 FROM t1, mydb2.t1" + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "db1.t1.i1" } ] } @@ -209,9 +209,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 23, - "fragment" : "SELECT mydb1.t1 FROM t1" + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "mydb1.t1" } ] } @@ -242,9 +242,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 23, - "fragment" : "SELECT t1 FROM mydb1.t1" + "startIndex" : 8, + "stopIndex" : 9, + "fragment" : "t1" } ] } @@ -274,9 +274,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 26, - "fragment" : "SELECT mydb1.t1.i1 FROM t1" + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "mydb1.t1.i1" } ] } 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 e1dbb7799e672..00c5f4c2d911b 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 @@ -653,8 +653,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd/MMMMM/yyyy'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd/MMMMM/yyyy'" } } @@ -669,8 +669,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd/MMMMM/yyyy'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd/MMMMM/yyyy'" } } @@ -685,8 +685,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd/MMMMM/yyyy'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd/MMMMM/yyyy'" } } diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out index da0a801c4badc..4b9a60a41d847 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out @@ -9,8 +9,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'GGGGG'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'GGGGG'" } } @@ -25,8 +25,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'yyyyyyy'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'yyyyyyy'" } } @@ -59,8 +59,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'MMMMM'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'MMMMM'" } } @@ -75,8 +75,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'LLLLL'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'LLLLL'" } } @@ -91,8 +91,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'EEEEE'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'EEEEE'" } } @@ -107,8 +107,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'FF'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'FF'" } } @@ -123,8 +123,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'ddd'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'ddd'" } } @@ -139,8 +139,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'DDDD'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'DDDD'" } } @@ -155,8 +155,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'HHH'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'HHH'" } } @@ -171,8 +171,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'hhh'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'hhh'" } } @@ -187,8 +187,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'kkk'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'kkk'" } } @@ -203,8 +203,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'KKK'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'KKK'" } } @@ -219,8 +219,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'mmm'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'mmm'" } } @@ -235,8 +235,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'sss'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'sss'" } } @@ -251,8 +251,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'SSSSSSSSSS'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'SSSSSSSSSS'" } } @@ -267,8 +267,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'aa'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'aa'" } } @@ -292,8 +292,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'zzzzz'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'zzzzz'" } } @@ -317,8 +317,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'ZZZZZZ'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'ZZZZZZ'" } } @@ -387,8 +387,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'Y'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'Y'" } } @@ -403,8 +403,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'w'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'w'" } } @@ -419,8 +419,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'W'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'W'" } } @@ -435,8 +435,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'u'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'u'" } } 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 decb0e6ac632c..bb363d7e059a7 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 @@ -1497,16 +1497,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 43, - "fragment" : "select timestamp'2011-11-11 11:11:11' + '1'" + "fragment" : "timestamp'2011-11-11 11:11:11' + '1'" } ] } @@ -1521,16 +1521,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"", "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 43, - "fragment" : "select '1' + timestamp'2011-11-11 11:11:11'" + "fragment" : "'1' + timestamp'2011-11-11 11:11:11'" } ] } @@ -1545,16 +1545,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\"", "left" : "\"TIMESTAMP\"", - "right" : "\"VOID\"" + "right" : "\"VOID\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 44, - "fragment" : "select timestamp'2011-11-11 11:11:11' + null" + "fragment" : "timestamp'2011-11-11 11:11:11' + null" } ] } @@ -1569,16 +1569,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\"", "left" : "\"VOID\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 44, - "fragment" : "select null + timestamp'2011-11-11 11:11:11'" + "fragment" : "null + timestamp'2011-11-11 11:11:11'" } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out index 79cf395abbabb..ef7626260b9f1 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out @@ -18,8 +18,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "PARSE_DATETIME_BY_NEW_PARSER", "messageParameters" : { - "datetime" : "'1'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "datetime" : "'1'" } } @@ -42,8 +42,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "PARSE_DATETIME_BY_NEW_PARSER", "messageParameters" : { - "datetime" : "'123'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "datetime" : "'123'" } } @@ -58,8 +58,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "PARSE_DATETIME_BY_NEW_PARSER", "messageParameters" : { - "datetime" : "'1'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "datetime" : "'1'" } } @@ -74,8 +74,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'yyyyyyy'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'yyyyyyy'" } } @@ -98,8 +98,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "PARSE_DATETIME_BY_NEW_PARSER", "messageParameters" : { - "datetime" : "'9'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "datetime" : "'9'" } } @@ -114,8 +114,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "PARSE_DATETIME_BY_NEW_PARSER", "messageParameters" : { - "datetime" : "'9'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "datetime" : "'9'" } } @@ -130,8 +130,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "PARSE_DATETIME_BY_NEW_PARSER", "messageParameters" : { - "datetime" : "'99'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "datetime" : "'99'" } } @@ -202,8 +202,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "PARSE_DATETIME_BY_NEW_PARSER", "messageParameters" : { - "datetime" : "'2018-366'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "datetime" : "'2018-366'" } } diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out index 45df378495ccb..5bb86a398c82a 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -474,9 +474,9 @@ org.apache.spark.sql.AnalysisException { "errorClass" : "FORBIDDEN_OPERATION", "messageParameters" : { - "statement" : "DESC PARTITION", + "objectName" : "`temp_v`", "objectType" : "TEMPORARY VIEW", - "objectName" : "`temp_v`" + "statement" : "DESC PARTITION" } } @@ -560,9 +560,9 @@ org.apache.spark.sql.AnalysisException { "errorClass" : "FORBIDDEN_OPERATION", "messageParameters" : { - "statement" : "DESC PARTITION", + "objectName" : "`v`", "objectType" : "VIEW", - "objectName" : "`v`" + "statement" : "DESC PARTITION" } } 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 f7b1a90ad4e10..b458ea6a2bf70 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out @@ -233,14 +233,7 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "expression" : "\"a\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 56, - "stopIndex" : 65, - "fragment" : "GROUP BY b" - } ] + } } @@ -726,14 +719,7 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "expression" : "\"a\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 65, - "stopIndex" : 78, - "fragment" : "GROUP BY a + 1" - } ] + } } diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out index 58bdf32e02eea..e21b79eb04697 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out @@ -165,8 +165,8 @@ org.apache.spark.sql.AnalysisException "errorClass" : "GROUP_BY_POS_REFERS_AGG_EXPR", "sqlState" : "42000", "messageParameters" : { - "index" : "3", - "aggExpr" : "sum(data.b) AS `sum(b)`" + "aggExpr" : "sum(data.b) AS `sum(b)`", + "index" : "3" }, "queryContext" : [ { "objectType" : "", @@ -188,8 +188,8 @@ org.apache.spark.sql.AnalysisException "errorClass" : "GROUP_BY_POS_REFERS_AGG_EXPR", "sqlState" : "42000", "messageParameters" : { - "index" : "3", - "aggExpr" : "(sum(data.b) + CAST(2 AS BIGINT)) AS `(sum(b) + 2)`" + "aggExpr" : "(sum(data.b) + CAST(2 AS BIGINT)) AS `(sum(b) + 2)`", + "index" : "3" }, "queryContext" : [ { "objectType" : "", @@ -446,8 +446,8 @@ org.apache.spark.sql.AnalysisException "errorClass" : "GROUP_BY_POS_REFERS_AGG_EXPR", "sqlState" : "42000", "messageParameters" : { - "index" : "3", - "aggExpr" : "count(1) AS `count(1)`" + "aggExpr" : "count(1) AS `count(1)`", + "index" : "3" }, "queryContext" : [ { "objectType" : "", @@ -492,8 +492,8 @@ org.apache.spark.sql.AnalysisException "errorClass" : "GROUP_BY_POS_REFERS_AGG_EXPR", "sqlState" : "42000", "messageParameters" : { - "index" : "3", - "aggExpr" : "count(1) AS `count(1)`" + "aggExpr" : "count(1) AS `count(1)`", + "index" : "3" }, "queryContext" : [ { "objectType" : "", 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 ad4106ff48997..3505c97bd4649 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -48,14 +48,7 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "expression" : "\"a\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 34, - "stopIndex" : 43, - "fragment" : "GROUP BY b" - } ] + } } @@ -125,14 +118,7 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "expression" : "\"a\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 38, - "stopIndex" : 51, - "fragment" : "GROUP BY a + 1" - } ] + } } @@ -201,9 +187,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 50, - "stopIndex" : 59, - "fragment" : "GROUP BY k" + "startIndex" : 22, + "stopIndex" : 33, + "fragment" : "non_existing" } ] } @@ -237,14 +223,7 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "expression" : "\"k\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 59, - "stopIndex" : 68, - "fragment" : "GROUP BY a" - } ] + } } @@ -273,9 +252,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 39, + "startIndex" : 48, "stopIndex" : 48, - "fragment" : "GROUP BY k" + "fragment" : "k" } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out b/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out index c024386da0c70..83b721373dae6 100644 --- a/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out @@ -171,14 +171,7 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "expression" : "\"c1\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 48, - "stopIndex" : 74, - "fragment" : "GROUP BY GROUPING SETS (())" - } ] + } } diff --git a/sql/core/src/test/resources/sql-tests/results/having.sql.out b/sql/core/src/test/resources/sql-tests/results/having.sql.out index 4f59e111069af..00b59827fde9b 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 @@ -39,16 +39,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(v = array(1))\"", "left" : "\"INT\"", - "right" : "\"ARRAY\"" + "right" : "\"ARRAY\"", + "sqlExpr" : "\"(v = array(1))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 44, "stopIndex" : 55, - "fragment" : "SELECT count(k) FROM hav GROUP BY v HAVING v = array(1)" + "fragment" : "v = array(1)" } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 29516de73149e..39fcd3aba41f2 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 @@ -187,16 +187,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(2 / INTERVAL '02' SECOND)\"", "left" : "\"DOUBLE\"", - "right" : "\"INTERVAL SECOND\"" + "right" : "\"INTERVAL SECOND\"", + "sqlExpr" : "\"(2 / INTERVAL '02' SECOND)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 30, - "fragment" : "select '2' / interval 2 second" + "fragment" : "'2' / interval 2 second" } ] } @@ -211,16 +211,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"", "left" : "\"DOUBLE\"", - "right" : "\"INTERVAL YEAR\"" + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 28, - "fragment" : "select '2' / interval 2 year" + "fragment" : "'2' / interval 2 year" } ] } @@ -321,16 +321,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"", "left" : "\"INT\"", - "right" : "\"INTERVAL YEAR\"" + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 28, - "fragment" : "select 2 / interval '2' year" + "fragment" : "2 / interval '2' year" } ] } @@ -345,16 +345,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(2 / INTERVAL '02' HOUR)\"", "left" : "\"INT\"", - "right" : "\"INTERVAL HOUR\"" + "right" : "\"INTERVAL HOUR\"", + "sqlExpr" : "\"(2 / INTERVAL '02' HOUR)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 28, - "fragment" : "select 2 / interval '2' hour" + "fragment" : "2 / interval '2' hour" } ] } @@ -369,16 +369,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(NULL / INTERVAL '2' YEAR)\"", "left" : "\"VOID\"", - "right" : "\"INTERVAL YEAR\"" + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(NULL / INTERVAL '2' YEAR)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 31, - "fragment" : "select null / interval '2' year" + "fragment" : "null / interval '2' year" } ] } @@ -393,16 +393,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(NULL / INTERVAL '02' HOUR)\"", "left" : "\"VOID\"", - "right" : "\"INTERVAL HOUR\"" + "right" : "\"INTERVAL HOUR\"", + "sqlExpr" : "\"(NULL / INTERVAL '02' HOUR)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 31, - "fragment" : "select null / interval '2' hour" + "fragment" : "null / interval '2' hour" } ] } @@ -1554,16 +1554,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3 year to month)\"", "left" : "\"INTERVAL YEAR\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3 year to month)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 388, - "fragment" : "select\n interval '2' year + '3-3 year to month',\n interval '2' year - '3 month',\n '3-2 year to month' + interval '2-2' year to month,\n '3 year' - interval '2-2' year to month,\n interval '99 11:22:33.123456789' day to second + '12:12 hour to second',\n interval '99 11:22:33.123456789' day to second - '12 hour',\n '4 day' + interval '10' day,\n '4 22 day to hour' - interval '10' day" + "startIndex" : 10, + "stopIndex" : 48, + "fragment" : "interval '2' year + '3-3 year to month'" } ] } @@ -1594,16 +1594,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3)\"", "left" : "\"INTERVAL YEAR\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 32, - "fragment" : "select interval '2' year + '3-3'" + "fragment" : "interval '2' year + '3-3'" } ] } @@ -1618,16 +1618,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '2' YEAR - 4)\"", "left" : "\"INTERVAL YEAR\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR - 4)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 30, - "fragment" : "select interval '2' year - '4'" + "fragment" : "interval '2' year - '4'" } ] } @@ -1666,16 +1666,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '2' YEAR + str)\"", "left" : "\"INTERVAL YEAR\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + str)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 49, - "fragment" : "select interval '2' year + str from interval_view" + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' year + str" } ] } @@ -1690,16 +1690,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '2' YEAR - str)\"", "left" : "\"INTERVAL YEAR\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR - str)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 49, - "fragment" : "select interval '2' year - str from interval_view" + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' year - str" } ] } @@ -1757,16 +1757,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '3' DAY - INTERVAL '2-2' YEAR TO MONTH)\"", "left" : "\"INTERVAL DAY\"", - "right" : "\"INTERVAL YEAR TO MONTH\"" + "right" : "\"INTERVAL YEAR TO MONTH\"", + "sqlExpr" : "\"(INTERVAL '3' DAY - INTERVAL '2-2' YEAR TO MONTH)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 54, - "fragment" : "select interval '3' day - interval '2-2' year to month" + "fragment" : "interval '3' day - interval '2-2' year to month" } ] } @@ -1790,16 +1790,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 + INTERVAL '2' MONTH)\"", "left" : "\"INT\"", - "right" : "\"INTERVAL MONTH\"" + "right" : "\"INTERVAL MONTH\"", + "sqlExpr" : "\"(1 + INTERVAL '2' MONTH)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 29, - "fragment" : "select 1 + interval '2' month" + "fragment" : "1 + interval '2' month" } ] } @@ -1823,16 +1823,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '2' MONTH - 1)\"", "left" : "\"INTERVAL MONTH\"", - "right" : "\"INT\"" + "right" : "\"INT\"", + "sqlExpr" : "\"(INTERVAL '2' MONTH - 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 29, - "fragment" : "select interval '2' month - 1" + "fragment" : "interval '2' month - 1" } ] } @@ -1974,8 +1974,8 @@ org.apache.spark.SparkArithmeticException "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "integer overflow", - "alternative" : "" + "alternative" : "", + "message" : "integer overflow" } } @@ -1990,8 +1990,8 @@ org.apache.spark.SparkArithmeticException "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "integer overflow", - "alternative" : " Use 'try_subtract' to tolerate overflow and return NULL instead." + "alternative" : " Use 'try_subtract' to tolerate overflow and return NULL instead.", + "message" : "integer overflow" } } @@ -2006,8 +2006,8 @@ org.apache.spark.SparkArithmeticException "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "integer overflow", - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead." + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "message" : "integer overflow" } } @@ -2242,8 +2242,8 @@ org.apache.spark.SparkArithmeticException "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "Interval value overflows after being divided by -1", - "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead." + "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.", + "message" : "Interval value overflows after being divided by -1" }, "queryContext" : [ { "objectType" : "", @@ -2265,8 +2265,8 @@ org.apache.spark.SparkArithmeticException "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "Interval value overflows after being divided by -1", - "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead." + "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.", + "message" : "Interval value overflows after being divided by -1" }, "queryContext" : [ { "objectType" : "", @@ -2322,8 +2322,8 @@ org.apache.spark.SparkArithmeticException "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "Interval value overflows after being divided by -1", - "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead." + "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.", + "message" : "Interval value overflows after being divided by -1" }, "queryContext" : [ { "objectType" : "", @@ -2345,8 +2345,8 @@ org.apache.spark.SparkArithmeticException "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "Interval value overflows after being divided by -1", - "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead." + "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.", + "message" : "Interval value overflows after being divided by -1" }, "queryContext" : [ { "objectType" : "", @@ -2694,16 +2694,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '1' MONTH > INTERVAL '20' DAY)\"", "left" : "\"INTERVAL MONTH\"", - "right" : "\"INTERVAL DAY\"" + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(INTERVAL '1' MONTH > INTERVAL '20' DAY)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 42, - "fragment" : "SELECT INTERVAL 1 MONTH > INTERVAL 20 DAYS" + "fragment" : "INTERVAL 1 MONTH > INTERVAL 20 DAYS" } ] } @@ -2718,16 +2718,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '1' DAY < 1)\"", "left" : "\"INTERVAL DAY\"", - "right" : "\"STRING\"" + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY < 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 29, - "fragment" : "SELECT INTERVAL '1' DAY < '1'" + "fragment" : "INTERVAL '1' DAY < '1'" } ] } @@ -2742,16 +2742,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '1' DAY = 1)\"", "left" : "\"INTERVAL DAY\"", - "right" : "\"STRING\"" + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY = 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 29, - "fragment" : "SELECT INTERVAL '1' DAY = '1'" + "fragment" : "INTERVAL '1' DAY = '1'" } ] } @@ -2766,16 +2766,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '1' DAY > 1)\"", "left" : "\"INTERVAL DAY\"", - "right" : "\"STRING\"" + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY > 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 29, - "fragment" : "SELECT INTERVAL '1' DAY > '1'" + "fragment" : "INTERVAL '1' DAY > '1'" } ] } @@ -2790,16 +2790,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 < INTERVAL '1' DAY)\"", "left" : "\"STRING\"", - "right" : "\"INTERVAL DAY\"" + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 < INTERVAL '1' DAY)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 29, - "fragment" : "SELECT '1' < INTERVAL '1' DAY" + "fragment" : "'1' < INTERVAL '1' DAY" } ] } @@ -2814,16 +2814,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 = INTERVAL '1' DAY)\"", "left" : "\"STRING\"", - "right" : "\"INTERVAL DAY\"" + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 = INTERVAL '1' DAY)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 29, - "fragment" : "SELECT '1' = INTERVAL '1' DAY" + "fragment" : "'1' = INTERVAL '1' DAY" } ] } @@ -2838,16 +2838,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 > INTERVAL '1' DAY)\"", "left" : "\"STRING\"", - "right" : "\"INTERVAL DAY\"" + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 > INTERVAL '1' DAY)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 29, - "fragment" : "SELECT '1' > INTERVAL '1' DAY" + "fragment" : "'1' > INTERVAL '1' DAY" } ] } @@ -2862,16 +2862,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '1' YEAR < 1)\"", "left" : "\"INTERVAL YEAR\"", - "right" : "\"STRING\"" + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR < 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 30, - "fragment" : "SELECT INTERVAL '1' YEAR < '1'" + "fragment" : "INTERVAL '1' YEAR < '1'" } ] } @@ -2886,16 +2886,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '1' YEAR = 1)\"", "left" : "\"INTERVAL YEAR\"", - "right" : "\"STRING\"" + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR = 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 30, - "fragment" : "SELECT INTERVAL '1' YEAR = '1'" + "fragment" : "INTERVAL '1' YEAR = '1'" } ] } @@ -2910,16 +2910,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '1' YEAR > 1)\"", "left" : "\"INTERVAL YEAR\"", - "right" : "\"STRING\"" + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR > 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 30, - "fragment" : "SELECT INTERVAL '1' YEAR > '1'" + "fragment" : "INTERVAL '1' YEAR > '1'" } ] } @@ -2934,16 +2934,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 < INTERVAL '1' YEAR)\"", "left" : "\"STRING\"", - "right" : "\"INTERVAL YEAR\"" + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 < INTERVAL '1' YEAR)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 30, - "fragment" : "SELECT '1' < INTERVAL '1' YEAR" + "fragment" : "'1' < INTERVAL '1' YEAR" } ] } @@ -2958,16 +2958,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 = INTERVAL '1' YEAR)\"", "left" : "\"STRING\"", - "right" : "\"INTERVAL YEAR\"" + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 = INTERVAL '1' YEAR)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 30, - "fragment" : "SELECT '1' = INTERVAL '1' YEAR" + "fragment" : "'1' = INTERVAL '1' YEAR" } ] } @@ -2982,16 +2982,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 > INTERVAL '1' YEAR)\"", "left" : "\"STRING\"", - "right" : "\"INTERVAL YEAR\"" + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 > INTERVAL '1' YEAR)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 30, - "fragment" : "SELECT '1' > INTERVAL '1' YEAR" + "fragment" : "'1' > INTERVAL '1' YEAR" } ] } @@ -3104,16 +3104,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(INTERVAL '1' MONTH div INTERVAL '-1' DAY)\"", "left" : "\"INTERVAL MONTH\"", - "right" : "\"INTERVAL DAY\"" + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(INTERVAL '1' MONTH div INTERVAL '-1' DAY)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 49, - "fragment" : "SELECT div(INTERVAL '1' MONTH, INTERVAL '-1' DAY)" + "fragment" : "div(INTERVAL '1' MONTH, INTERVAL '-1' DAY)" } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out index f68d18175f923..df0b7dbad14cc 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 @@ -288,9 +288,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 86, - "fragment" : "SELECT * FROM t1 JOIN LATERAL (SELECT t1.c1 AS a, t2.c1 AS b) s JOIN t2 ON s.b = t2.c1" + "startIndex" : 51, + "stopIndex" : 55, + "fragment" : "t2.c1" } ] } @@ -420,9 +420,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 76, - "fragment" : "SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.c1 + t2.c1))" + "startIndex" : 62, + "stopIndex" : 66, + "fragment" : "t1.c1" } ] } @@ -443,9 +443,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 74, - "fragment" : "SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1), LATERAL (SELECT c2))" + "startIndex" : 71, + "stopIndex" : 72, + "fragment" : "c2" } ] } @@ -485,9 +485,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 80, - "fragment" : "SELECT * FROM t1, LATERAL (SELECT c1, (SELECT SUM(c2) FROM t2 WHERE c1 = t1.c1))" + "startIndex" : 74, + "stopIndex" : 78, + "fragment" : "t1.c1" } ] } 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 ca079e4add048..97b219589a4f2 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 @@ -340,8 +340,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "PARSE_DATETIME_BY_NEW_PARSER", "messageParameters" : { - "datetime" : "'02-29'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "datetime" : "'02-29'" } } @@ -359,8 +359,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "PARSE_DATETIME_BY_NEW_PARSER", "messageParameters" : { - "datetime" : "'02-29'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "datetime" : "'02-29'" } } 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 bfcf63a318697..4d8847b6d0d6c 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 @@ -240,9 +240,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 54, - "fragment" : "SELECT nt2.k FROM (SELECT * FROM nt1 natural join nt2)" + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "nt2.k" } ] } 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 5eb70900d465a..b25362aba1efd 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 @@ -240,9 +240,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 62, "stopIndex" : 113, - "fragment" : "SELECT * FROM (\n SELECT course, earnings FROM courseSales\n)\nPIVOT (\n sum(earnings)\n FOR year IN (2012, 2013)\n)" + "fragment" : "PIVOT (\n sum(earnings)\n FOR year IN (2012, 2013)\n)" } ] } @@ -329,9 +329,9 @@ org.apache.spark.sql.AnalysisException "errorClass" : "PIVOT_VALUE_DATA_TYPE_MISMATCH", "sqlState" : "42000", "messageParameters" : { + "pivotType" : "struct", "value" : "dotNET", - "valueType" : "string", - "pivotType" : "struct" + "valueType" : "string" } } @@ -357,9 +357,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 75, - "fragment" : "SELECT * FROM courseSales\nPIVOT (\n sum(earnings)\n FOR year IN (s, 2013)\n)" + "startIndex" : 66, + "stopIndex" : 66, + "fragment" : "s" } ] } 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 f4a3688348e2e..ecbb871fdd2d3 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 @@ -506,8 +506,8 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 95, - "fragment" : "select\n (select max((select i.unique2 from tenk1 i where i.unique1 = o.unique1)))\nfrom tenk1 o" + "startIndex" : 71, + "stopIndex" : 79, + "fragment" : "o.unique1" } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out index 2ef04e1669232..371ae63424f48 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out @@ -57,10 +57,10 @@ org.apache.spark.SparkRuntimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'test'", "sourceType" : "\"STRING\"", - "targetType" : "\"BOOLEAN\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BOOLEAN\"" }, "queryContext" : [ { "objectType" : "", @@ -90,10 +90,10 @@ org.apache.spark.SparkRuntimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'foo'", "sourceType" : "\"STRING\"", - "targetType" : "\"BOOLEAN\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BOOLEAN\"" }, "queryContext" : [ { "objectType" : "", @@ -131,10 +131,10 @@ org.apache.spark.SparkRuntimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'yeah'", "sourceType" : "\"STRING\"", - "targetType" : "\"BOOLEAN\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BOOLEAN\"" }, "queryContext" : [ { "objectType" : "", @@ -172,10 +172,10 @@ org.apache.spark.SparkRuntimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'nay'", "sourceType" : "\"STRING\"", - "targetType" : "\"BOOLEAN\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BOOLEAN\"" }, "queryContext" : [ { "objectType" : "", @@ -197,10 +197,10 @@ org.apache.spark.SparkRuntimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'on'", "sourceType" : "\"STRING\"", - "targetType" : "\"BOOLEAN\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BOOLEAN\"" }, "queryContext" : [ { "objectType" : "", @@ -222,10 +222,10 @@ org.apache.spark.SparkRuntimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'off'", "sourceType" : "\"STRING\"", - "targetType" : "\"BOOLEAN\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BOOLEAN\"" }, "queryContext" : [ { "objectType" : "", @@ -247,10 +247,10 @@ org.apache.spark.SparkRuntimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'of'", "sourceType" : "\"STRING\"", - "targetType" : "\"BOOLEAN\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BOOLEAN\"" }, "queryContext" : [ { "objectType" : "", @@ -272,10 +272,10 @@ org.apache.spark.SparkRuntimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'o'", "sourceType" : "\"STRING\"", - "targetType" : "\"BOOLEAN\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BOOLEAN\"" }, "queryContext" : [ { "objectType" : "", @@ -297,10 +297,10 @@ org.apache.spark.SparkRuntimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'on_'", "sourceType" : "\"STRING\"", - "targetType" : "\"BOOLEAN\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BOOLEAN\"" }, "queryContext" : [ { "objectType" : "", @@ -322,10 +322,10 @@ org.apache.spark.SparkRuntimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'off_'", "sourceType" : "\"STRING\"", - "targetType" : "\"BOOLEAN\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BOOLEAN\"" }, "queryContext" : [ { "objectType" : "", @@ -355,10 +355,10 @@ org.apache.spark.SparkRuntimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'11'", "sourceType" : "\"STRING\"", - "targetType" : "\"BOOLEAN\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BOOLEAN\"" }, "queryContext" : [ { "objectType" : "", @@ -388,10 +388,10 @@ org.apache.spark.SparkRuntimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'000'", "sourceType" : "\"STRING\"", - "targetType" : "\"BOOLEAN\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BOOLEAN\"" }, "queryContext" : [ { "objectType" : "", @@ -413,10 +413,10 @@ org.apache.spark.SparkRuntimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "''", "sourceType" : "\"STRING\"", - "targetType" : "\"BOOLEAN\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BOOLEAN\"" }, "queryContext" : [ { "objectType" : "", @@ -535,10 +535,10 @@ org.apache.spark.SparkRuntimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "' tru e '", "sourceType" : "\"STRING\"", - "targetType" : "\"BOOLEAN\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BOOLEAN\"" }, "queryContext" : [ { "objectType" : "", @@ -560,10 +560,10 @@ org.apache.spark.SparkRuntimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "''", "sourceType" : "\"STRING\"", - "targetType" : "\"BOOLEAN\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BOOLEAN\"" }, "queryContext" : [ { "objectType" : "", 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 26e05a444d25f..fcdd42551d1d3 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 @@ -79,9 +79,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 109, - "fragment" : "CREATE VIEW key_dependent_view_no_cols AS\n SELECT FROM view_base_table GROUP BY key HAVING length(data) > 0" + "startIndex" : 53, + "stopIndex" : 56, + "fragment" : "FROM" } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out index 673252707709c..a2fb4276f6e37 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out @@ -97,10 +97,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'N A N'", "sourceType" : "\"STRING\"", - "targetType" : "\"FLOAT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"FLOAT\"" }, "queryContext" : [ { "objectType" : "", @@ -122,10 +122,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'NaN x'", "sourceType" : "\"STRING\"", - "targetType" : "\"FLOAT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"FLOAT\"" }, "queryContext" : [ { "objectType" : "", @@ -147,10 +147,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "' INFINITY x'", "sourceType" : "\"STRING\"", - "targetType" : "\"FLOAT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"FLOAT\"" }, "queryContext" : [ { "objectType" : "", @@ -196,10 +196,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'nan'", "sourceType" : "\"STRING\"", - "targetType" : "\"DECIMAL(10,0)\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"DECIMAL(10,0)\"" }, "queryContext" : [ { "objectType" : "", @@ -393,10 +393,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "CAST_OVERFLOW", "sqlState" : "22005", "messageParameters" : { - "value" : "2.14748365E9", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "sourceType" : "\"FLOAT\"", "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "value" : "2.14748365E9" } } @@ -419,10 +419,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "CAST_OVERFLOW", "sqlState" : "22005", "messageParameters" : { - "value" : "-2.1474839E9", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "sourceType" : "\"FLOAT\"", "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "value" : "-2.1474839E9" } } @@ -461,10 +461,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "CAST_OVERFLOW", "sqlState" : "22005", "messageParameters" : { - "value" : "-9.22338E18", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "sourceType" : "\"FLOAT\"", "targetType" : "\"BIGINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "value" : "-9.22338E18" } } diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out index a8dbeb13b2d89..a9f7bdc925c4a 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out @@ -129,10 +129,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'N A N'", "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"DOUBLE\"" }, "queryContext" : [ { "objectType" : "", @@ -154,10 +154,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'NaN x'", "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"DOUBLE\"" }, "queryContext" : [ { "objectType" : "", @@ -179,10 +179,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "' INFINITY x'", "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"DOUBLE\"" }, "queryContext" : [ { "objectType" : "", @@ -228,10 +228,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'nan'", "sourceType" : "\"STRING\"", - "targetType" : "\"DECIMAL(10,0)\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"DECIMAL(10,0)\"" }, "queryContext" : [ { "objectType" : "", @@ -898,10 +898,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "CAST_OVERFLOW", "sqlState" : "22005", "messageParameters" : { - "value" : "-9.22337203685478E18D", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "sourceType" : "\"DOUBLE\"", "targetType" : "\"BIGINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "value" : "-9.22337203685478E18D" } } diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out index 5b9edcf763fd4..16c18c86f2919 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out @@ -201,9 +201,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "integer overflow", "alternative" : " Use 'try_multiply' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" }, "queryContext" : [ { "objectType" : "", @@ -236,9 +236,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "integer overflow", "alternative" : " Use 'try_multiply' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" }, "queryContext" : [ { "objectType" : "", @@ -271,9 +271,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "integer overflow", "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" }, "queryContext" : [ { "objectType" : "", @@ -307,9 +307,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "integer overflow", "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" }, "queryContext" : [ { "objectType" : "", @@ -343,9 +343,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "integer overflow", "alternative" : " Use 'try_subtract' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" }, "queryContext" : [ { "objectType" : "", @@ -379,9 +379,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "integer overflow", "alternative" : " Use 'try_subtract' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out index e4056ac2e6c8c..d1974a1c0d8ee 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out @@ -393,9 +393,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "long overflow", "alternative" : " Use 'try_multiply' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" }, "queryContext" : [ { "objectType" : "", @@ -737,10 +737,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "CAST_OVERFLOW", "sqlState" : "22005", "messageParameters" : { - "value" : "4567890123456789L", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "sourceType" : "\"BIGINT\"", "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "value" : "4567890123456789L" } } @@ -763,10 +763,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "CAST_OVERFLOW", "sqlState" : "22005", "messageParameters" : { - "value" : "4567890123456789L", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "sourceType" : "\"BIGINT\"", "targetType" : "\"SMALLINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "value" : "4567890123456789L" } } @@ -809,10 +809,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "CAST_OVERFLOW", "sqlState" : "22005", "messageParameters" : { - "value" : "9.223372036854776E20D", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "sourceType" : "\"DOUBLE\"", "targetType" : "\"BIGINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "value" : "9.223372036854776E20D" } } @@ -890,10 +890,10 @@ org.apache.spark.SparkArithmeticException "errorClass" : "CAST_OVERFLOW", "sqlState" : "22005", "messageParameters" : { - "value" : "-9223372036854775808L", + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "sourceType" : "\"BIGINT\"", "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "value" : "-9223372036854775808L" } } @@ -908,9 +908,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "long overflow", "alternative" : " Use 'try_multiply' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" }, "queryContext" : [ { "objectType" : "", @@ -948,9 +948,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "long overflow", "alternative" : " Use 'try_multiply' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" }, "queryContext" : [ { "objectType" : "", @@ -988,9 +988,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "long overflow", "alternative" : " Use 'try_multiply' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" }, "queryContext" : [ { "objectType" : "", 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 de2a939c8a549..df9b4ec209535 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 @@ -3256,9 +3256,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 78, "stopIndex" : 81, - "fragment" : "select * from\n int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = y.f1" + "fragment" : "y.f1" } ] } @@ -3290,9 +3290,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 67, - "fragment" : "select t1.uunique1 from\n tenk1 t1 join tenk2 t2 on t1.two = t2.two" + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "t1.uunique1" } ] } @@ -3315,9 +3315,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 67, - "fragment" : "select t2.uunique1 from\n tenk1 t1 join tenk2 t2 on t1.two = t2.two" + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "t2.uunique1" } ] } @@ -3340,9 +3340,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 64, - "fragment" : "select uunique1 from\n tenk1 t1 join tenk2 t2 on t1.two = t2.two" + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "uunique1" } ] } @@ -3554,9 +3554,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 48, - "fragment" : "select f1,g from int4_tbl a, (select f1 as g) ss" + "startIndex" : 38, + "stopIndex" : 39, + "fragment" : "f1" } ] } @@ -3577,9 +3577,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 50, - "fragment" : "select f1,g from int4_tbl a, (select a.f1 as g) ss" + "startIndex" : 38, + "stopIndex" : 41, + "fragment" : "a.f1" } ] } @@ -3600,9 +3600,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 58, - "fragment" : "select f1,g from int4_tbl a cross join (select f1 as g) ss" + "startIndex" : 48, + "stopIndex" : 49, + "fragment" : "f1" } ] } @@ -3623,9 +3623,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "select f1,g from int4_tbl a cross join (select a.f1 as g) ss" + "startIndex" : 48, + "stopIndex" : 51, + "fragment" : "a.f1" } ] } 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 d717c5f7e677b..2e9a827ce5555 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 @@ -160,9 +160,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 45, - "fragment" : "SELECT 1 AS one FROM test_having HAVING a > 1" + "startIndex" : 41, + "stopIndex" : 41, + "fragment" : "a" } ] } 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 b5d1df99934c7..e6fb22e8d4967 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 @@ -130,9 +130,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 53, + "startIndex" : 62, "stopIndex" : 62, - "fragment" : "ORDER BY b" + "fragment" : "b" } ] } @@ -364,9 +364,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 53, + "startIndex" : 62, "stopIndex" : 62, - "fragment" : "ORDER BY b" + "fragment" : "b" } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out index 1067fb6ee18d8..4867410806d3b 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out @@ -66,10 +66,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'four: 2'", "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BIGINT\"" }, "queryContext" : [ { "objectType" : "", @@ -91,10 +91,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'four: 2'", "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"BIGINT\"" }, "queryContext" : [ { "objectType" : "", @@ -312,9 +312,9 @@ org.apache.spark.sql.AnalysisException "errorClass" : "INVALID_PARAMETER_VALUE", "sqlState" : "22023", "messageParameters" : { - "parameter" : "strfmt", + "expected" : "expects %1$, %2$ and so on, but got %0$.", "functionName" : "`format_string`", - "expected" : "expects %1$, %2$ and so on, but got %0$." + "parameter" : "strfmt" }, "queryContext" : [ { "objectType" : "", 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 75c3fc1f53b16..354c6b26cea44 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 @@ -592,9 +592,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 56, - "stopIndex" : 74, - "fragment" : "ORDER BY q2 LIMIT 1" + "startIndex" : 65, + "stopIndex" : 66, + "fragment" : "q2" } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out index 50027cb4a902d..f70d694080387 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out @@ -226,9 +226,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "long overflow", "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" }, "queryContext" : [ { "objectType" : "", @@ -249,9 +249,9 @@ org.apache.spark.SparkArithmeticException "errorClass" : "ARITHMETIC_OVERFLOW", "sqlState" : "22003", "messageParameters" : { - "message" : "long overflow", "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "spark.sql.ansi.enabled" + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" }, "queryContext" : [ { "objectType" : "", @@ -489,10 +489,10 @@ org.apache.spark.SparkNumberFormatException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'NaN'", "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"INT\"" }, "queryContext" : [ { "objectType" : "", 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 03b8f59ab8b7c..df14c281a5a91 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 @@ -44,9 +44,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 43, - "fragment" : "SELECT `(a)?+.+` FROM testData2 WHERE a = 1" + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "`(a)?+.+`" } ] } @@ -68,9 +68,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 47, - "fragment" : "SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1" + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "t.`(a)?+.+`" } ] } @@ -92,9 +92,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 41, - "fragment" : "SELECT `(a|b)` FROM testData2 WHERE a = 2" + "startIndex" : 8, + "stopIndex" : 14, + "fragment" : "`(a|b)`" } ] } @@ -116,9 +116,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 45, - "fragment" : "SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2" + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "`(a|b)?+.+`" } ] } @@ -140,9 +140,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 38, - "fragment" : "SELECT SUM(`(a|b)?+.+`) FROM testData2" + "startIndex" : 12, + "stopIndex" : 22, + "fragment" : "`(a|b)?+.+`" } ] } @@ -164,9 +164,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 32, - "fragment" : "SELECT SUM(`(a)`) FROM testData2" + "startIndex" : 12, + "stopIndex" : 16, + "fragment" : "`(a)`" } ] } @@ -399,9 +399,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 30, + "startIndex" : 39, "stopIndex" : 43, - "fragment" : "GROUP BY `(a)`" + "fragment" : "`(a)`" } ] } @@ -423,8 +423,8 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 30, + "startIndex" : 39, "stopIndex" : 47, - "fragment" : "GROUP BY `(a)?+.+`" + "fragment" : "`(a)?+.+`" } ] } 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 65e1e31ae7cf3..9ef1802738a1f 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 @@ -135,9 +135,9 @@ org.apache.spark.SparkRuntimeException "errorClass" : "INVALID_PARAMETER_VALUE", "sqlState" : "22023", "messageParameters" : { - "parameter" : "regexp", + "expected" : "(?l)", "functionName" : "`regexp_extract`", - "expected" : "(?l)" + "parameter" : "regexp" } } @@ -278,9 +278,9 @@ org.apache.spark.SparkRuntimeException "errorClass" : "INVALID_PARAMETER_VALUE", "sqlState" : "22023", "messageParameters" : { - "parameter" : "regexp", + "expected" : "], [", "functionName" : "`regexp_extract_all`", - "expected" : "], [" + "parameter" : "regexp" } } @@ -585,8 +585,8 @@ org.apache.spark.SparkRuntimeException "errorClass" : "INVALID_PARAMETER_VALUE", "sqlState" : "22023", "messageParameters" : { - "parameter" : "regexp", + "expected" : ") ?", "functionName" : "`regexp_instr`", - "expected" : ") ?" + "parameter" : "regexp" } } 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 a8ad802dd9899..8502c60ce6e27 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 @@ -1119,10 +1119,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "' ab cdef= = '", "fmt" : "'BASE64'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "' ab cdef= = '", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1146,10 +1146,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "'a'", "fmt" : "'BASE64'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "'a'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1163,10 +1163,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "'a?'", "fmt" : "'BASE64'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "'a?'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1180,10 +1180,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "'abcde'", "fmt" : "'BASE64'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "'abcde'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1197,10 +1197,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "'abcd='", "fmt" : "'BASE64'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "'abcd='", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1214,10 +1214,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "'a==='", "fmt" : "'BASE64'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "'a==='", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1231,10 +1231,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "'ab==f'", "fmt" : "'BASE64'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "'ab==f'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1321,10 +1321,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "'GG'", "fmt" : "'HEX'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "'GG'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1338,10 +1338,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "'01 AF'", "fmt" : "'HEX'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "'01 AF'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1363,10 +1363,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "' ab cdef= = '", "fmt" : "'BASE64'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "' ab cdef= = '", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } @@ -1380,10 +1380,10 @@ org.apache.spark.SparkIllegalArgumentException { "errorClass" : "CONVERSION_INVALID_INPUT", "messageParameters" : { - "str" : "' ab cdef= = '", "fmt" : "'HEX'", - "targetType" : "\"BINARY\"", - "suggestion" : "`try_to_binary`" + "str" : "' ab cdef= = '", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" } } 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 68dfc470edc62..3193e3a023637 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 @@ -145,8 +145,8 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 87, - "fragment" : "SELECT t1.t1a\nFROM t1\nJOIN t1_copy\nON EXISTS (SELECT 1 FROM t2 WHERE t2a > t1a)" + "startIndex" : 84, + "stopIndex" : 86, + "fragment" : "t1a" } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out index cd54af2ae9967..b38d742b03f93 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 @@ -68,9 +68,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 52, - "fragment" : "SELECT a AS col1, b AS col2 FROM testData AS t(c, d)" + "startIndex" : 8, + "stopIndex" : 8, + "fragment" : "a" } ] } 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 3a8a9684c66c4..d9042f0bbb2cd 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 @@ -669,16 +669,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 43, - "fragment" : "select timestamp'2011-11-11 11:11:11' + '1'" + "fragment" : "timestamp'2011-11-11 11:11:11' + '1'" } ] } @@ -693,16 +693,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"", "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 43, - "fragment" : "select '1' + timestamp'2011-11-11 11:11:11'" + "fragment" : "'1' + timestamp'2011-11-11 11:11:11'" } ] } @@ -717,16 +717,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\"", "left" : "\"TIMESTAMP\"", - "right" : "\"VOID\"" + "right" : "\"VOID\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 44, - "fragment" : "select timestamp'2011-11-11 11:11:11' + null" + "fragment" : "timestamp'2011-11-11 11:11:11' + null" } ] } @@ -741,16 +741,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\"", "left" : "\"VOID\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 44, - "fragment" : "select null + timestamp'2011-11-11 11:11:11'" + "fragment" : "null + timestamp'2011-11-11 11:11:11'" } ] } @@ -788,8 +788,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'yyyy-MM-dd GGGGG'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'yyyy-MM-dd GGGGG'" } } @@ -804,8 +804,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd MM yyyy EEEEEE'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd MM yyyy EEEEEE'" } } @@ -820,8 +820,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd MM yyyy EEEEE'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd MM yyyy EEEEE'" } } @@ -836,8 +836,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd MM yyyy EEEEE'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd MM yyyy EEEEE'" } } @@ -852,8 +852,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd/MMMMM/yyyy'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd/MMMMM/yyyy'" } } @@ -868,8 +868,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd/MMMMM/yyyy'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd/MMMMM/yyyy'" } } 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 446687e63aea5..269b6d5ff0c23 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 @@ -339,10 +339,10 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CAST_INVALID_INPUT", "sqlState" : "42000", "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", "expression" : "'1'", "sourceType" : "\"STRING\"", - "targetType" : "\"TIMESTAMP_NTZ\"", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "targetType" : "\"TIMESTAMP_NTZ\"" }, "queryContext" : [ { "objectType" : "", @@ -362,8 +362,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text '2019-10-06 10:11:12.' could not be parsed at index 20", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2019-10-06 10:11:12.' could not be parsed at index 20" } } @@ -434,8 +434,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text '2019-10-06 10:11:12.1234567PST' could not be parsed, unparsed text found at index 26", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2019-10-06 10:11:12.1234567PST' could not be parsed, unparsed text found at index 26" } } @@ -458,8 +458,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text '223456 2019-10-06 10:11:12.123456PST' could not be parsed at index 27", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '223456 2019-10-06 10:11:12.123456PST' could not be parsed at index 27" } } @@ -530,8 +530,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 7", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 7" } } @@ -546,8 +546,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 9", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 9" } } @@ -626,8 +626,8 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "42000", "messageParameters" : { - "message" : "Invalid date 'February 29' as '1970' is not a leap year", - "ansiConfig" : "\"spark.sql.ansi.enabled\"" + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Invalid date 'February 29' as '1970' is not a leap year" } } @@ -738,16 +738,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_WRONG_TYPE", "messageParameters" : { - "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + 1)\"", + "actualDataType" : "\"TIMESTAMP_NTZ\"", "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", - "actualDataType" : "\"TIMESTAMP_NTZ\"" + "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 43, - "fragment" : "select timestamp'2011-11-11 11:11:11' + '1'" + "fragment" : "timestamp'2011-11-11 11:11:11' + '1'" } ] } @@ -762,16 +762,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_WRONG_TYPE", "messageParameters" : { - "sqlExpr" : "\"(1 + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"", + "actualDataType" : "\"TIMESTAMP_NTZ\"", "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", - "actualDataType" : "\"TIMESTAMP_NTZ\"" + "sqlExpr" : "\"(1 + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 43, - "fragment" : "select '1' + timestamp'2011-11-11 11:11:11'" + "fragment" : "'1' + timestamp'2011-11-11 11:11:11'" } ] } @@ -786,16 +786,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + NULL)\"", "left" : "\"TIMESTAMP_NTZ\"", - "right" : "\"VOID\"" + "right" : "\"VOID\"", + "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + NULL)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 44, - "fragment" : "select timestamp'2011-11-11 11:11:11' + null" + "fragment" : "timestamp'2011-11-11 11:11:11' + null" } ] } @@ -810,16 +810,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(NULL + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"", "left" : "\"VOID\"", - "right" : "\"TIMESTAMP_NTZ\"" + "right" : "\"TIMESTAMP_NTZ\"", + "sqlExpr" : "\"(NULL + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 44, - "fragment" : "select null + timestamp'2011-11-11 11:11:11'" + "fragment" : "null + timestamp'2011-11-11 11:11:11'" } ] } @@ -884,8 +884,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd MM yyyy EEEEE'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd MM yyyy EEEEE'" } } 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 0661df238e37d..b1ff61b4c9dd8 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 @@ -669,16 +669,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + 1)\"", "left" : "\"TIMESTAMP_NTZ\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 43, - "fragment" : "select timestamp'2011-11-11 11:11:11' + '1'" + "fragment" : "timestamp'2011-11-11 11:11:11' + '1'" } ] } @@ -693,16 +693,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"", "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP_NTZ\"" + "right" : "\"TIMESTAMP_NTZ\"", + "sqlExpr" : "\"(1 + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 43, - "fragment" : "select '1' + timestamp'2011-11-11 11:11:11'" + "fragment" : "'1' + timestamp'2011-11-11 11:11:11'" } ] } @@ -717,16 +717,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + NULL)\"", "left" : "\"TIMESTAMP_NTZ\"", - "right" : "\"VOID\"" + "right" : "\"VOID\"", + "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + NULL)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 44, - "fragment" : "select timestamp'2011-11-11 11:11:11' + null" + "fragment" : "timestamp'2011-11-11 11:11:11' + null" } ] } @@ -741,16 +741,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(NULL + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"", "left" : "\"VOID\"", - "right" : "\"TIMESTAMP_NTZ\"" + "right" : "\"TIMESTAMP_NTZ\"", + "sqlExpr" : "\"(NULL + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 8, "stopIndex" : 44, - "fragment" : "select null + timestamp'2011-11-11 11:11:11'" + "fragment" : "null + timestamp'2011-11-11 11:11:11'" } ] } @@ -815,8 +815,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd MM yyyy EEEEE'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd MM yyyy EEEEE'" } } diff --git a/sql/core/src/test/resources/sql-tests/results/try_datetime_functions.sql.out b/sql/core/src/test/resources/sql-tests/results/try_datetime_functions.sql.out index ab2172f56de86..c49637869e1fc 100644 --- a/sql/core/src/test/resources/sql-tests/results/try_datetime_functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/try_datetime_functions.sql.out @@ -49,7 +49,7 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'dd MM yyyy EEEEEE'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'dd MM yyyy EEEEEE'" } } 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 a4a3bedf59887..2912754249912 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 @@ -81,16 +81,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(true = CAST(1 AS BINARY))\"", "left" : "\"BOOLEAN\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(true = CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 40, - "fragment" : "SELECT true = cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "true = cast('1' as binary)" } ] } @@ -113,16 +113,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(true = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"BOOLEAN\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(true = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 63, - "fragment" : "SELECT true = cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "true = cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -137,16 +137,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(true = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"BOOLEAN\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(true = CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 56, - "fragment" : "SELECT true = cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "true = cast('2017-12-11 09:30:00' as date)" } ] } @@ -225,16 +225,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(true <=> CAST(1 AS BINARY))\"", "left" : "\"BOOLEAN\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(true <=> CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 42, - "fragment" : "SELECT true <=> cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "true <=> cast('1' as binary)" } ] } @@ -257,16 +257,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(true <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"BOOLEAN\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(true <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 65, - "fragment" : "SELECT true <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "true <=> cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -281,16 +281,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(true <=> CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"BOOLEAN\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(true <=> CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 58, - "fragment" : "SELECT true <=> cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "true <=> cast('2017-12-11 09:30:00' as date)" } ] } @@ -369,16 +369,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) = true)\"", "left" : "\"BINARY\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = true)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 40, - "fragment" : "SELECT cast('1' as binary) = true FROM t" + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast('1' as binary) = true" } ] } @@ -401,16 +401,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = true)\"", "left" : "\"TIMESTAMP\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = true)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 63, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) = true FROM t" + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = true" } ] } @@ -425,16 +425,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = true)\"", "left" : "\"DATE\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = true)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 56, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) = true FROM t" + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "cast('2017-12-11 09:30:00' as date) = true" } ] } @@ -513,16 +513,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) <=> true)\"", "left" : "\"BINARY\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <=> true)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 42, - "fragment" : "SELECT cast('1' as binary) <=> true FROM t" + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "cast('1' as binary) <=> true" } ] } @@ -545,16 +545,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> true)\"", "left" : "\"TIMESTAMP\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> true)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 65, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> true FROM t" + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> true" } ] } @@ -569,16 +569,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> true)\"", "left" : "\"DATE\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> true)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 58, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <=> true FROM t" + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> true" } ] } @@ -657,16 +657,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(false = CAST(0 AS BINARY))\"", "left" : "\"BOOLEAN\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(false = CAST(0 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 41, - "fragment" : "SELECT false = cast('0' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "false = cast('0' as binary)" } ] } @@ -689,16 +689,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(false = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"BOOLEAN\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(false = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 64, - "fragment" : "SELECT false = cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "false = cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -713,16 +713,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(false = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"BOOLEAN\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(false = CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 57, - "fragment" : "SELECT false = cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "false = cast('2017-12-11 09:30:00' as date)" } ] } @@ -801,16 +801,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(false <=> CAST(0 AS BINARY))\"", "left" : "\"BOOLEAN\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(false <=> CAST(0 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 43, - "fragment" : "SELECT false <=> cast('0' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "false <=> cast('0' as binary)" } ] } @@ -833,16 +833,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(false <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"BOOLEAN\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(false <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 66, - "fragment" : "SELECT false <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "false <=> cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -857,16 +857,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(false <=> CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"BOOLEAN\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(false <=> CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 59, - "fragment" : "SELECT false <=> cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "false <=> cast('2017-12-11 09:30:00' as date)" } ] } @@ -945,16 +945,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(0 AS BINARY) = false)\"", "left" : "\"BINARY\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS BINARY) = false)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 41, - "fragment" : "SELECT cast('0' as binary) = false FROM t" + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast('0' as binary) = false" } ] } @@ -977,16 +977,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = false)\"", "left" : "\"TIMESTAMP\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = false)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 64, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) = false FROM t" + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = false" } ] } @@ -1001,16 +1001,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = false)\"", "left" : "\"DATE\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = false)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 57, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) = false FROM t" + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "cast('2017-12-11 09:30:00' as date) = false" } ] } @@ -1089,16 +1089,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(0 AS BINARY) <=> false)\"", "left" : "\"BINARY\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS BINARY) <=> false)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 43, - "fragment" : "SELECT cast('0' as binary) <=> false FROM t" + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "cast('0' as binary) <=> false" } ] } @@ -1121,16 +1121,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> false)\"", "left" : "\"TIMESTAMP\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> false)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 66, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> false FROM t" + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> false" } ] } @@ -1145,15 +1145,15 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> false)\"", "left" : "\"DATE\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> false)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 59, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <=> false FROM t" + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> false" } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out index 0f485014ef3f7..4b9973c53cae5 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 @@ -241,16 +241,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast('1' as binary) + cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) + cast(1 as decimal(3, 0))" } ] } @@ -265,16 +265,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast('1' as binary) + cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) + cast(1 as decimal(5, 0))" } ] } @@ -289,16 +289,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) + cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) + cast(1 as decimal(10, 0))" } ] } @@ -313,16 +313,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) + cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) + cast(1 as decimal(20, 0))" } ] } @@ -337,16 +337,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 83, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(3, 0))" } ] } @@ -361,16 +361,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 83, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(5, 0))" } ] } @@ -385,16 +385,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(10, 0))" } ] } @@ -409,16 +409,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(20, 0))" } ] } @@ -725,16 +725,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(3, 0)) + cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) + cast('1' as binary)" } ] } @@ -749,16 +749,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(5, 0)) + cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) + cast('1' as binary)" } ] } @@ -773,16 +773,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(10, 0)) + cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) + cast('1' as binary)" } ] } @@ -797,16 +797,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(20, 0)) + cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) + cast('1' as binary)" } ] } @@ -821,16 +821,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(3, 0)) + cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(3, 0)) + cast(1 as boolean)" } ] } @@ -845,16 +845,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(5, 0)) + cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(5, 0)) + cast(1 as boolean)" } ] } @@ -869,16 +869,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(10, 0)) + cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(10, 0)) + cast(1 as boolean)" } ] } @@ -893,16 +893,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(20, 0)) + cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(20, 0)) + cast(1 as boolean)" } ] } @@ -917,16 +917,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -941,16 +941,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -965,16 +965,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -989,16 +989,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -1273,16 +1273,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast('1' as binary) - cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) - cast(1 as decimal(3, 0))" } ] } @@ -1297,16 +1297,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast('1' as binary) - cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) - cast(1 as decimal(5, 0))" } ] } @@ -1321,16 +1321,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) - cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) - cast(1 as decimal(10, 0))" } ] } @@ -1345,16 +1345,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) - cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) - cast(1 as decimal(20, 0))" } ] } @@ -1697,16 +1697,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(3, 0)) - cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) - cast('1' as binary)" } ] } @@ -1721,16 +1721,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(5, 0)) - cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) - cast('1' as binary)" } ] } @@ -1745,16 +1745,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(10, 0)) - cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) - cast('1' as binary)" } ] } @@ -1769,16 +1769,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(20, 0)) - cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) - cast('1' as binary)" } ] } @@ -1793,16 +1793,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(3, 0)) - cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(3, 0)) - cast(1 as boolean)" } ] } @@ -1817,16 +1817,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(5, 0)) - cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(5, 0)) - cast(1 as boolean)" } ] } @@ -1841,16 +1841,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(10, 0)) - cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(10, 0)) - cast(1 as boolean)" } ] } @@ -1865,16 +1865,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(20, 0)) - cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(20, 0)) - cast(1 as boolean)" } ] } @@ -2185,16 +2185,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast('1' as binary) * cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) * cast(1 as decimal(3, 0))" } ] } @@ -2209,16 +2209,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast('1' as binary) * cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) * cast(1 as decimal(5, 0))" } ] } @@ -2233,16 +2233,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) * cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) * cast(1 as decimal(10, 0))" } ] } @@ -2257,16 +2257,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) * cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) * cast(1 as decimal(20, 0))" } ] } @@ -2281,16 +2281,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 83, - "fragment" : "SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(3, 0))" } ] } @@ -2305,16 +2305,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 83, - "fragment" : "SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(5, 0))" } ] } @@ -2329,16 +2329,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(10, 0))" } ] } @@ -2353,16 +2353,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(20, 0))" } ] } @@ -2377,16 +2377,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 76, - "fragment" : "SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(3, 0))" } ] } @@ -2401,16 +2401,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 76, - "fragment" : "SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(5, 0))" } ] } @@ -2425,16 +2425,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(10, 0))" } ] } @@ -2449,16 +2449,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(20, 0))" } ] } @@ -2729,16 +2729,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(3, 0)) * cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) * cast('1' as binary)" } ] } @@ -2753,16 +2753,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(5, 0)) * cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) * cast('1' as binary)" } ] } @@ -2777,16 +2777,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(10, 0)) * cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) * cast('1' as binary)" } ] } @@ -2801,16 +2801,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(20, 0)) * cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) * cast('1' as binary)" } ] } @@ -2825,16 +2825,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(3, 0)) * cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(3, 0)) * cast(1 as boolean)" } ] } @@ -2849,16 +2849,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(5, 0)) * cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(5, 0)) * cast(1 as boolean)" } ] } @@ -2873,16 +2873,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(10, 0)) * cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(10, 0)) * cast(1 as boolean)" } ] } @@ -2897,16 +2897,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(20, 0)) * cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(20, 0)) * cast(1 as boolean)" } ] } @@ -2921,16 +2921,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00.0' as timestamp)" } ] } @@ -2945,16 +2945,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00.0' as timestamp)" } ] } @@ -2969,16 +2969,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00.0' as timestamp)" } ] } @@ -2993,16 +2993,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00.0' as timestamp)" } ] } @@ -3017,16 +3017,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00' as date)" } ] } @@ -3041,16 +3041,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00' as date)" } ] } @@ -3065,16 +3065,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00' as date)" } ] } @@ -3089,16 +3089,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00' as date)" } ] } @@ -3337,16 +3337,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast('1' as binary) / cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) / cast(1 as decimal(3, 0))" } ] } @@ -3361,16 +3361,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast('1' as binary) / cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) / cast(1 as decimal(5, 0))" } ] } @@ -3385,16 +3385,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) / cast(1 as decimal(10, 0))" } ] } @@ -3409,16 +3409,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) / cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) / cast(1 as decimal(20, 0))" } ] } @@ -3433,16 +3433,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 83, - "fragment" : "SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(3, 0))" } ] } @@ -3457,16 +3457,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 83, - "fragment" : "SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(5, 0))" } ] } @@ -3481,16 +3481,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0))" } ] } @@ -3505,16 +3505,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(20, 0))" } ] } @@ -3529,16 +3529,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 76, - "fragment" : "SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(3, 0))" } ] } @@ -3553,16 +3553,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 76, - "fragment" : "SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(5, 0))" } ] } @@ -3577,16 +3577,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(10, 0))" } ] } @@ -3601,16 +3601,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(20, 0))" } ] } @@ -3881,16 +3881,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(3, 0)) / cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) / cast('1' as binary)" } ] } @@ -3905,16 +3905,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(5, 0)) / cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) / cast('1' as binary)" } ] } @@ -3929,16 +3929,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) / cast('1' as binary)" } ] } @@ -3953,16 +3953,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(20, 0)) / cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) / cast('1' as binary)" } ] } @@ -3977,16 +3977,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(3, 0)) / cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(3, 0)) / cast(1 as boolean)" } ] } @@ -4001,16 +4001,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(5, 0)) / cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(5, 0)) / cast(1 as boolean)" } ] } @@ -4025,16 +4025,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(10, 0)) / cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(10, 0)) / cast(1 as boolean)" } ] } @@ -4049,16 +4049,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(20, 0)) / cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(20, 0)) / cast(1 as boolean)" } ] } @@ -4073,16 +4073,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00.0' as timestamp)" } ] } @@ -4097,16 +4097,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00.0' as timestamp)" } ] } @@ -4121,16 +4121,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00.0' as timestamp)" } ] } @@ -4145,16 +4145,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00.0' as timestamp)" } ] } @@ -4169,16 +4169,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00' as date)" } ] } @@ -4193,16 +4193,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00' as date)" } ] } @@ -4217,16 +4217,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00' as date)" } ] } @@ -4241,16 +4241,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00' as date)" } ] } @@ -4489,16 +4489,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast('1' as binary) % cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) % cast(1 as decimal(3, 0))" } ] } @@ -4513,16 +4513,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast('1' as binary) % cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) % cast(1 as decimal(5, 0))" } ] } @@ -4537,16 +4537,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) % cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) % cast(1 as decimal(10, 0))" } ] } @@ -4561,16 +4561,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) % cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) % cast(1 as decimal(20, 0))" } ] } @@ -4585,16 +4585,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 83, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(3, 0))" } ] } @@ -4609,16 +4609,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 83, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(5, 0))" } ] } @@ -4633,16 +4633,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(10, 0))" } ] } @@ -4657,16 +4657,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(20, 0))" } ] } @@ -4681,16 +4681,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 76, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(3, 0))" } ] } @@ -4705,16 +4705,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 76, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(5, 0))" } ] } @@ -4729,16 +4729,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(10, 0))" } ] } @@ -4753,16 +4753,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(20, 0))" } ] } @@ -5033,16 +5033,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(3, 0)) % cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) % cast('1' as binary)" } ] } @@ -5057,16 +5057,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(5, 0)) % cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) % cast('1' as binary)" } ] } @@ -5081,16 +5081,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(10, 0)) % cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) % cast('1' as binary)" } ] } @@ -5105,16 +5105,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(20, 0)) % cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) % cast('1' as binary)" } ] } @@ -5129,16 +5129,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(3, 0)) % cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(3, 0)) % cast(1 as boolean)" } ] } @@ -5153,16 +5153,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(5, 0)) % cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(5, 0)) % cast(1 as boolean)" } ] } @@ -5177,16 +5177,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(10, 0)) % cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(10, 0)) % cast(1 as boolean)" } ] } @@ -5201,16 +5201,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(20, 0)) % cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(20, 0)) % cast(1 as boolean)" } ] } @@ -5225,16 +5225,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -5249,16 +5249,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -5273,16 +5273,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -5297,16 +5297,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -5321,16 +5321,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00' as date)" } ] } @@ -5345,16 +5345,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00' as date)" } ] } @@ -5369,16 +5369,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00' as date)" } ] } @@ -5393,16 +5393,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00' as date)" } ] } @@ -5641,16 +5641,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 65, - "fragment" : "SELECT pmod(cast('1' as binary), cast(1 as decimal(3, 0))) FROM t" + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "pmod(cast('1' as binary), cast(1 as decimal(3, 0)))" } ] } @@ -5665,16 +5665,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 65, - "fragment" : "SELECT pmod(cast('1' as binary), cast(1 as decimal(5, 0))) FROM t" + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "pmod(cast('1' as binary), cast(1 as decimal(5, 0)))" } ] } @@ -5689,16 +5689,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 66, - "fragment" : "SELECT pmod(cast('1' as binary), cast(1 as decimal(10, 0))) FROM t" + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "pmod(cast('1' as binary), cast(1 as decimal(10, 0)))" } ] } @@ -5713,16 +5713,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 66, - "fragment" : "SELECT pmod(cast('1' as binary), cast(1 as decimal(20, 0))) FROM t" + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "pmod(cast('1' as binary), cast(1 as decimal(20, 0)))" } ] } @@ -5737,16 +5737,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 88, - "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(3, 0))) FROM t" + "startIndex" : 8, + "stopIndex" : 81, + "fragment" : "pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(3, 0)))" } ] } @@ -5761,16 +5761,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 88, - "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(5, 0))) FROM t" + "startIndex" : 8, + "stopIndex" : 81, + "fragment" : "pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(5, 0)))" } ] } @@ -5785,16 +5785,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 89, - "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(10, 0))) FROM t" + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(10, 0)))" } ] } @@ -5809,16 +5809,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 89, - "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(20, 0))) FROM t" + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(20, 0)))" } ] } @@ -5833,16 +5833,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 81, - "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(3, 0))) FROM t" + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(3, 0)))" } ] } @@ -5857,16 +5857,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 81, - "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(5, 0))) FROM t" + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(5, 0)))" } ] } @@ -5881,16 +5881,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 82, - "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(10, 0))) FROM t" + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(10, 0)))" } ] } @@ -5905,16 +5905,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 82, - "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(20, 0))) FROM t" + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(20, 0)))" } ] } @@ -6185,16 +6185,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 66, - "fragment" : "SELECT pmod(cast(1 as decimal(3, 0)) , cast('1' as binary)) FROM t" + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "pmod(cast(1 as decimal(3, 0)) , cast('1' as binary))" } ] } @@ -6209,16 +6209,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 66, - "fragment" : "SELECT pmod(cast(1 as decimal(5, 0)) , cast('1' as binary)) FROM t" + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "pmod(cast(1 as decimal(5, 0)) , cast('1' as binary))" } ] } @@ -6233,16 +6233,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 66, - "fragment" : "SELECT pmod(cast(1 as decimal(10, 0)), cast('1' as binary)) FROM t" + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "pmod(cast(1 as decimal(10, 0)), cast('1' as binary))" } ] } @@ -6257,16 +6257,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 66, - "fragment" : "SELECT pmod(cast(1 as decimal(20, 0)), cast('1' as binary)) FROM t" + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "pmod(cast(1 as decimal(20, 0)), cast('1' as binary))" } ] } @@ -6281,16 +6281,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 65, - "fragment" : "SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as boolean)) FROM t" + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "pmod(cast(1 as decimal(3, 0)) , cast(1 as boolean))" } ] } @@ -6305,16 +6305,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 65, - "fragment" : "SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as boolean)) FROM t" + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "pmod(cast(1 as decimal(5, 0)) , cast(1 as boolean))" } ] } @@ -6329,16 +6329,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 65, - "fragment" : "SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as boolean)) FROM t" + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "pmod(cast(1 as decimal(10, 0)), cast(1 as boolean))" } ] } @@ -6353,16 +6353,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 65, - "fragment" : "SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as boolean)) FROM t" + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "pmod(cast(1 as decimal(20, 0)), cast(1 as boolean))" } ] } @@ -6377,16 +6377,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 89, - "fragment" : "SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00.0' as timestamp)) FROM t" + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00.0' as timestamp))" } ] } @@ -6401,16 +6401,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 89, - "fragment" : "SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00.0' as timestamp)) FROM t" + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00.0' as timestamp))" } ] } @@ -6425,16 +6425,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 89, - "fragment" : "SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t" + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp))" } ] } @@ -6449,16 +6449,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 89, - "fragment" : "SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t" + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00.0' as timestamp))" } ] } @@ -6473,16 +6473,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 82, - "fragment" : "SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00' as date)) FROM t" + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00' as date))" } ] } @@ -6497,16 +6497,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 82, - "fragment" : "SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00' as date)) FROM t" + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00' as date))" } ] } @@ -6521,16 +6521,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 82, - "fragment" : "SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM t" + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date))" } ] } @@ -6545,16 +6545,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 82, - "fragment" : "SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00' as date)) FROM t" + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00' as date))" } ] } @@ -6793,16 +6793,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast('1' as binary) = cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) = cast(1 as decimal(3, 0))" } ] } @@ -6817,16 +6817,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast('1' as binary) = cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) = cast(1 as decimal(5, 0))" } ] } @@ -6841,16 +6841,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) = cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) = cast(1 as decimal(10, 0))" } ] } @@ -6865,16 +6865,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) = cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) = cast(1 as decimal(20, 0))" } ] } @@ -6889,16 +6889,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 83, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(3, 0))" } ] } @@ -6913,16 +6913,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 83, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(5, 0))" } ] } @@ -6937,16 +6937,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(10, 0))" } ] } @@ -6961,16 +6961,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(20, 0))" } ] } @@ -6985,16 +6985,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 76, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(3, 0))" } ] } @@ -7009,16 +7009,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 76, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(5, 0))" } ] } @@ -7033,16 +7033,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(10, 0))" } ] } @@ -7057,16 +7057,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(20, 0))" } ] } @@ -7337,16 +7337,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(3, 0)) = cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) = cast('1' as binary)" } ] } @@ -7361,16 +7361,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(5, 0)) = cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) = cast('1' as binary)" } ] } @@ -7385,16 +7385,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(10, 0)) = cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) = cast('1' as binary)" } ] } @@ -7409,16 +7409,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(20, 0)) = cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) = cast('1' as binary)" } ] } @@ -7465,16 +7465,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -7489,16 +7489,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -7513,16 +7513,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -7537,16 +7537,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -7561,16 +7561,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00' as date)" } ] } @@ -7585,16 +7585,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00' as date)" } ] } @@ -7609,16 +7609,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00' as date)" } ] } @@ -7633,16 +7633,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00' as date)" } ] } @@ -7881,16 +7881,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('1' as binary) <=> cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) <=> cast(1 as decimal(3, 0))" } ] } @@ -7905,16 +7905,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('1' as binary) <=> cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) <=> cast(1 as decimal(5, 0))" } ] } @@ -7929,16 +7929,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 63, - "fragment" : "SELECT cast('1' as binary) <=> cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('1' as binary) <=> cast(1 as decimal(10, 0))" } ] } @@ -7953,16 +7953,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 63, - "fragment" : "SELECT cast('1' as binary) <=> cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('1' as binary) <=> cast(1 as decimal(20, 0))" } ] } @@ -7977,16 +7977,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(3, 0))" } ] } @@ -8001,16 +8001,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(5, 0))" } ] } @@ -8025,16 +8025,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 86, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(10, 0))" } ] } @@ -8049,16 +8049,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 86, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(20, 0))" } ] } @@ -8073,16 +8073,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(3, 0))" } ] } @@ -8097,16 +8097,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(5, 0))" } ] } @@ -8121,16 +8121,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 79, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(10, 0))" } ] } @@ -8145,16 +8145,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 79, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(20, 0))" } ] } @@ -8425,16 +8425,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 63, - "fragment" : "SELECT cast(1 as decimal(3, 0)) <=> cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast(1 as decimal(3, 0)) <=> cast('1' as binary)" } ] } @@ -8449,16 +8449,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 63, - "fragment" : "SELECT cast(1 as decimal(5, 0)) <=> cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast(1 as decimal(5, 0)) <=> cast('1' as binary)" } ] } @@ -8473,16 +8473,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 63, - "fragment" : "SELECT cast(1 as decimal(10, 0)) <=> cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast(1 as decimal(10, 0)) <=> cast('1' as binary)" } ] } @@ -8497,16 +8497,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 63, - "fragment" : "SELECT cast(1 as decimal(20, 0)) <=> cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast(1 as decimal(20, 0)) <=> cast('1' as binary)" } ] } @@ -8553,16 +8553,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 86, - "fragment" : "SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -8577,16 +8577,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 86, - "fragment" : "SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -8601,16 +8601,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 86, - "fragment" : "SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -8625,16 +8625,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 86, - "fragment" : "SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -8649,16 +8649,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 79, - "fragment" : "SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00' as date)" } ] } @@ -8673,16 +8673,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 79, - "fragment" : "SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00' as date)" } ] } @@ -8697,16 +8697,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 79, - "fragment" : "SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00' as date)" } ] } @@ -8721,16 +8721,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 79, - "fragment" : "SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00' as date)" } ] } @@ -8969,16 +8969,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast('1' as binary) < cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) < cast(1 as decimal(3, 0))" } ] } @@ -8993,16 +8993,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast('1' as binary) < cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) < cast(1 as decimal(5, 0))" } ] } @@ -9017,16 +9017,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) < cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) < cast(1 as decimal(10, 0))" } ] } @@ -9041,16 +9041,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) < cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) < cast(1 as decimal(20, 0))" } ] } @@ -9065,16 +9065,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 83, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(3, 0))" } ] } @@ -9089,16 +9089,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 83, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(5, 0))" } ] } @@ -9113,16 +9113,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(10, 0))" } ] } @@ -9137,16 +9137,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(20, 0))" } ] } @@ -9161,16 +9161,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 76, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(3, 0))" } ] } @@ -9185,16 +9185,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 76, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(5, 0))" } ] } @@ -9209,16 +9209,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(10, 0))" } ] } @@ -9233,16 +9233,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(20, 0))" } ] } @@ -9513,16 +9513,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(3, 0)) < cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) < cast('1' as binary)" } ] } @@ -9537,16 +9537,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(5, 0)) < cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) < cast('1' as binary)" } ] } @@ -9561,16 +9561,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(10, 0)) < cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) < cast('1' as binary)" } ] } @@ -9585,16 +9585,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(20, 0)) < cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) < cast('1' as binary)" } ] } @@ -9609,16 +9609,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(3, 0)) < cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(3, 0)) < cast(1 as boolean)" } ] } @@ -9633,16 +9633,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(5, 0)) < cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(5, 0)) < cast(1 as boolean)" } ] } @@ -9657,16 +9657,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(10, 0)) < cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(10, 0)) < cast(1 as boolean)" } ] } @@ -9681,16 +9681,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(20, 0)) < cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(20, 0)) < cast(1 as boolean)" } ] } @@ -9705,16 +9705,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -9729,16 +9729,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -9753,16 +9753,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -9777,16 +9777,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -9801,16 +9801,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00' as date)" } ] } @@ -9825,16 +9825,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00' as date)" } ] } @@ -9849,16 +9849,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00' as date)" } ] } @@ -9873,16 +9873,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00' as date)" } ] } @@ -10121,16 +10121,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) <= cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) <= cast(1 as decimal(3, 0))" } ] } @@ -10145,16 +10145,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) <= cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) <= cast(1 as decimal(5, 0))" } ] } @@ -10169,16 +10169,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('1' as binary) <= cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) <= cast(1 as decimal(10, 0))" } ] } @@ -10193,16 +10193,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('1' as binary) <= cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) <= cast(1 as decimal(20, 0))" } ] } @@ -10217,16 +10217,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(3, 0))" } ] } @@ -10241,16 +10241,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(5, 0))" } ] } @@ -10265,16 +10265,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(10, 0))" } ] } @@ -10289,16 +10289,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(20, 0))" } ] } @@ -10313,16 +10313,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(3, 0))" } ] } @@ -10337,16 +10337,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(5, 0))" } ] } @@ -10361,16 +10361,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(10, 0))" } ] } @@ -10385,16 +10385,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(20, 0))" } ] } @@ -10665,16 +10665,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast(1 as decimal(3, 0)) <= cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(3, 0)) <= cast('1' as binary)" } ] } @@ -10689,16 +10689,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast(1 as decimal(5, 0)) <= cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(5, 0)) <= cast('1' as binary)" } ] } @@ -10713,16 +10713,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast(1 as decimal(10, 0)) <= cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(10, 0)) <= cast('1' as binary)" } ] } @@ -10737,16 +10737,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast(1 as decimal(20, 0)) <= cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(20, 0)) <= cast('1' as binary)" } ] } @@ -10761,16 +10761,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(3, 0)) <= cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) <= cast(1 as boolean)" } ] } @@ -10785,16 +10785,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(5, 0)) <= cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) <= cast(1 as boolean)" } ] } @@ -10809,16 +10809,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(10, 0)) <= cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) <= cast(1 as boolean)" } ] } @@ -10833,16 +10833,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(20, 0)) <= cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) <= cast(1 as boolean)" } ] } @@ -10857,16 +10857,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -10881,16 +10881,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -10905,16 +10905,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -10929,16 +10929,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -10953,16 +10953,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00' as date)" } ] } @@ -10977,16 +10977,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00' as date)" } ] } @@ -11001,16 +11001,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00' as date)" } ] } @@ -11025,16 +11025,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00' as date)" } ] } @@ -11273,16 +11273,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast('1' as binary) > cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) > cast(1 as decimal(3, 0))" } ] } @@ -11297,16 +11297,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast('1' as binary) > cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) > cast(1 as decimal(5, 0))" } ] } @@ -11321,16 +11321,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) > cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) > cast(1 as decimal(10, 0))" } ] } @@ -11345,16 +11345,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) > cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) > cast(1 as decimal(20, 0))" } ] } @@ -11369,16 +11369,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 83, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(3, 0))" } ] } @@ -11393,16 +11393,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 83, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(5, 0))" } ] } @@ -11417,16 +11417,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(10, 0))" } ] } @@ -11441,16 +11441,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(20, 0))" } ] } @@ -11465,16 +11465,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 76, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(3, 0))" } ] } @@ -11489,16 +11489,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 76, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(5, 0))" } ] } @@ -11513,16 +11513,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(10, 0))" } ] } @@ -11537,16 +11537,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(20, 0))" } ] } @@ -11817,16 +11817,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(3, 0)) > cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) > cast('1' as binary)" } ] } @@ -11841,16 +11841,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(5, 0)) > cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) > cast('1' as binary)" } ] } @@ -11865,16 +11865,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(10, 0)) > cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) > cast('1' as binary)" } ] } @@ -11889,16 +11889,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(20, 0)) > cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) > cast('1' as binary)" } ] } @@ -11913,16 +11913,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(3, 0)) > cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(3, 0)) > cast(1 as boolean)" } ] } @@ -11937,16 +11937,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(5, 0)) > cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(5, 0)) > cast(1 as boolean)" } ] } @@ -11961,16 +11961,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(10, 0)) > cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(10, 0)) > cast(1 as boolean)" } ] } @@ -11985,16 +11985,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(20, 0)) > cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(20, 0)) > cast(1 as boolean)" } ] } @@ -12009,16 +12009,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -12033,16 +12033,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -12057,16 +12057,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -12081,16 +12081,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -12105,16 +12105,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00' as date)" } ] } @@ -12129,16 +12129,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00' as date)" } ] } @@ -12153,16 +12153,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00' as date)" } ] } @@ -12177,16 +12177,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00' as date)" } ] } @@ -12425,16 +12425,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) >= cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) >= cast(1 as decimal(3, 0))" } ] } @@ -12449,16 +12449,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) >= cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) >= cast(1 as decimal(5, 0))" } ] } @@ -12473,16 +12473,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('1' as binary) >= cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) >= cast(1 as decimal(10, 0))" } ] } @@ -12497,16 +12497,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('1' as binary) >= cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) >= cast(1 as decimal(20, 0))" } ] } @@ -12521,16 +12521,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(3, 0))" } ] } @@ -12545,16 +12545,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(5, 0))" } ] } @@ -12569,16 +12569,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(10, 0))" } ] } @@ -12593,16 +12593,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(20, 0))" } ] } @@ -12617,16 +12617,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(3, 0))" } ] } @@ -12641,16 +12641,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(5, 0))" } ] } @@ -12665,16 +12665,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(10, 0))" } ] } @@ -12689,16 +12689,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(20, 0))" } ] } @@ -12969,16 +12969,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast(1 as decimal(3, 0)) >= cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(3, 0)) >= cast('1' as binary)" } ] } @@ -12993,16 +12993,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast(1 as decimal(5, 0)) >= cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(5, 0)) >= cast('1' as binary)" } ] } @@ -13017,16 +13017,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast(1 as decimal(10, 0)) >= cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(10, 0)) >= cast('1' as binary)" } ] } @@ -13041,16 +13041,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast(1 as decimal(20, 0)) >= cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(20, 0)) >= cast('1' as binary)" } ] } @@ -13065,16 +13065,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(3, 0)) >= cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) >= cast(1 as boolean)" } ] } @@ -13089,16 +13089,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(5, 0)) >= cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) >= cast(1 as boolean)" } ] } @@ -13113,16 +13113,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(10, 0)) >= cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) >= cast(1 as boolean)" } ] } @@ -13137,16 +13137,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(20, 0)) >= cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) >= cast(1 as boolean)" } ] } @@ -13161,16 +13161,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -13185,16 +13185,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -13209,16 +13209,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -13233,16 +13233,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -13257,16 +13257,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00' as date)" } ] } @@ -13281,16 +13281,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00' as date)" } ] } @@ -13305,16 +13305,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00' as date)" } ] } @@ -13329,16 +13329,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00' as date)" } ] } @@ -13577,16 +13577,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) <> cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) <> cast(1 as decimal(3, 0))" } ] } @@ -13601,16 +13601,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) <> cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) <> cast(1 as decimal(5, 0))" } ] } @@ -13625,16 +13625,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('1' as binary) <> cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) <> cast(1 as decimal(10, 0))" } ] } @@ -13649,16 +13649,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('1' as binary) <> cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) <> cast(1 as decimal(20, 0))" } ] } @@ -13673,16 +13673,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(3, 0))" } ] } @@ -13697,16 +13697,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(5, 0))" } ] } @@ -13721,16 +13721,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(10, 0))" } ] } @@ -13745,16 +13745,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(20, 0))" } ] } @@ -13769,16 +13769,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(3,0)\"" + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(3,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(3, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(3, 0))" } ] } @@ -13793,16 +13793,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(5,0)\"" + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(5,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(5, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(5, 0))" } ] } @@ -13817,16 +13817,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(10, 0))" } ] } @@ -13841,16 +13841,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(20,0)\"" + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(20,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(20, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(20, 0))" } ] } @@ -14121,16 +14121,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast(1 as decimal(3, 0)) <> cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(3, 0)) <> cast('1' as binary)" } ] } @@ -14145,16 +14145,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast(1 as decimal(5, 0)) <> cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(5, 0)) <> cast('1' as binary)" } ] } @@ -14169,16 +14169,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast(1 as decimal(10, 0)) <> cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(10, 0)) <> cast('1' as binary)" } ] } @@ -14193,16 +14193,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast(1 as decimal(20, 0)) <> cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(20, 0)) <> cast('1' as binary)" } ] } @@ -14249,16 +14249,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -14273,16 +14273,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -14297,16 +14297,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -14321,16 +14321,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 85, - "fragment" : "SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -14345,16 +14345,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00' as date)" } ] } @@ -14369,16 +14369,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00' as date)" } ] } @@ -14393,16 +14393,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00' as date)" } ] } @@ -14417,15 +14417,15 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00' as date)" } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out index e404130cb2510..6a889ff469811 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 @@ -81,16 +81,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(1 AS BINARY))\"", "left" : "\"TINYINT\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 54, - "fragment" : "SELECT cast(1 as tinyint) / cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast(1 as tinyint) / cast('1' as binary)" } ] } @@ -105,16 +105,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(1 AS BOOLEAN))\"", "left" : "\"TINYINT\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 53, - "fragment" : "SELECT cast(1 as tinyint) / cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(1 as tinyint) / cast(1 as boolean)" } ] } @@ -129,16 +129,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"TINYINT\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as tinyint) / cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -153,16 +153,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"TINYINT\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 70, - "fragment" : "SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 63, + "fragment" : "cast(1 as tinyint) / cast('2017-12-11 09:30:00' as date)" } ] } @@ -241,16 +241,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(1 AS BINARY))\"", "left" : "\"SMALLINT\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 55, - "fragment" : "SELECT cast(1 as smallint) / cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "cast(1 as smallint) / cast('1' as binary)" } ] } @@ -265,16 +265,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(1 AS BOOLEAN))\"", "left" : "\"SMALLINT\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 54, - "fragment" : "SELECT cast(1 as smallint) / cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast(1 as smallint) / cast(1 as boolean)" } ] } @@ -289,16 +289,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"SMALLINT\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as smallint) / cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -313,16 +313,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"SMALLINT\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 71, - "fragment" : "SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "cast(1 as smallint) / cast('2017-12-11 09:30:00' as date)" } ] } @@ -401,16 +401,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS INT) / CAST(1 AS BINARY))\"", "left" : "\"INT\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS INT) / CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 50, - "fragment" : "SELECT cast(1 as int) / cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "cast(1 as int) / cast('1' as binary)" } ] } @@ -425,16 +425,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS INT) / CAST(1 AS BOOLEAN))\"", "left" : "\"INT\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS INT) / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 49, - "fragment" : "SELECT cast(1 as int) / cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "cast(1 as int) / cast(1 as boolean)" } ] } @@ -449,16 +449,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS INT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"INT\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS INT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 73, - "fragment" : "SELECT cast(1 as int) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 66, + "fragment" : "cast(1 as int) / cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -473,16 +473,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS INT) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"INT\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS INT) / CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 66, - "fragment" : "SELECT cast(1 as int) / cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "cast(1 as int) / cast('2017-12-11 09:30:00' as date)" } ] } @@ -561,16 +561,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(1 AS BINARY))\"", "left" : "\"BIGINT\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 53, - "fragment" : "SELECT cast(1 as bigint) / cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(1 as bigint) / cast('1' as binary)" } ] } @@ -585,16 +585,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(1 AS BOOLEAN))\"", "left" : "\"BIGINT\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 52, - "fragment" : "SELECT cast(1 as bigint) / cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as bigint) / cast(1 as boolean)" } ] } @@ -609,16 +609,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"BIGINT\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 76, - "fragment" : "SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast(1 as bigint) / cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -633,16 +633,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"BIGINT\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 69, - "fragment" : "SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "cast(1 as bigint) / cast('2017-12-11 09:30:00' as date)" } ] } @@ -721,16 +721,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(1 AS BINARY))\"", "left" : "\"FLOAT\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 52, - "fragment" : "SELECT cast(1 as float) / cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as float) / cast('1' as binary)" } ] } @@ -745,16 +745,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(1 AS BOOLEAN))\"", "left" : "\"FLOAT\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 51, - "fragment" : "SELECT cast(1 as float) / cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "cast(1 as float) / cast(1 as boolean)" } ] } @@ -769,16 +769,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"FLOAT\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 75, - "fragment" : "SELECT cast(1 as float) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 68, + "fragment" : "cast(1 as float) / cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -793,16 +793,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"FLOAT\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 68, - "fragment" : "SELECT cast(1 as float) / cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "cast(1 as float) / cast('2017-12-11 09:30:00' as date)" } ] } @@ -881,16 +881,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(1 AS BINARY))\"", "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 53, - "fragment" : "SELECT cast(1 as double) / cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(1 as double) / cast('1' as binary)" } ] } @@ -905,16 +905,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(1 AS BOOLEAN))\"", "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 52, - "fragment" : "SELECT cast(1 as double) / cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as double) / cast(1 as boolean)" } ] } @@ -929,16 +929,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 76, - "fragment" : "SELECT cast(1 as double) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast(1 as double) / cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -953,16 +953,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DOUBLE\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 69, - "fragment" : "SELECT cast(1 as double) / cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "cast(1 as double) / cast('2017-12-11 09:30:00' as date)" } ] } @@ -1041,16 +1041,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) / cast('1' as binary)" } ] } @@ -1065,16 +1065,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as decimal(10, 0)) / cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(10, 0)) / cast(1 as boolean)" } ] } @@ -1089,16 +1089,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -1113,16 +1113,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00' as date)" } ] } @@ -1201,16 +1201,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(1 AS BINARY))\"", "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 53, - "fragment" : "SELECT cast(1 as string) / cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(1 as string) / cast('1' as binary)" } ] } @@ -1225,16 +1225,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(1 AS BOOLEAN))\"", "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 52, - "fragment" : "SELECT cast(1 as string) / cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as string) / cast(1 as boolean)" } ] } @@ -1249,16 +1249,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 76, - "fragment" : "SELECT cast(1 as string) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast(1 as string) / cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -1273,16 +1273,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DOUBLE\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 69, - "fragment" : "SELECT cast(1 as string) / cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "cast(1 as string) / cast('2017-12-11 09:30:00' as date)" } ] } @@ -1297,16 +1297,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS TINYINT))\"", "left" : "\"BINARY\"", - "right" : "\"TINYINT\"" + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS TINYINT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 54, - "fragment" : "SELECT cast('1' as binary) / cast(1 as tinyint) FROM t" + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast('1' as binary) / cast(1 as tinyint)" } ] } @@ -1321,16 +1321,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS SMALLINT))\"", "left" : "\"BINARY\"", - "right" : "\"SMALLINT\"" + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS SMALLINT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 55, - "fragment" : "SELECT cast('1' as binary) / cast(1 as smallint) FROM t" + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "cast('1' as binary) / cast(1 as smallint)" } ] } @@ -1345,16 +1345,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS INT))\"", "left" : "\"BINARY\"", - "right" : "\"INT\"" + "right" : "\"INT\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS INT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 50, - "fragment" : "SELECT cast('1' as binary) / cast(1 as int) FROM t" + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "cast('1' as binary) / cast(1 as int)" } ] } @@ -1369,16 +1369,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS BIGINT))\"", "left" : "\"BINARY\"", - "right" : "\"BIGINT\"" + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS BIGINT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 53, - "fragment" : "SELECT cast('1' as binary) / cast(1 as bigint) FROM t" + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast('1' as binary) / cast(1 as bigint)" } ] } @@ -1393,16 +1393,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS FLOAT))\"", "left" : "\"BINARY\"", - "right" : "\"FLOAT\"" + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS FLOAT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 52, - "fragment" : "SELECT cast('1' as binary) / cast(1 as float) FROM t" + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast('1' as binary) / cast(1 as float)" } ] } @@ -1417,16 +1417,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DOUBLE))\"", "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DOUBLE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 53, - "fragment" : "SELECT cast('1' as binary) / cast(1 as double) FROM t" + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast('1' as binary) / cast(1 as double)" } ] } @@ -1441,16 +1441,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 61, - "fragment" : "SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) / cast(1 as decimal(10, 0))" } ] } @@ -1465,16 +1465,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS STRING))\"", "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS STRING))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 53, - "fragment" : "SELECT cast('1' as binary) / cast(1 as string) FROM t" + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast('1' as binary) / cast(1 as string)" } ] } @@ -1489,16 +1489,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_WRONG_TYPE", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS BINARY))\"", + "actualDataType" : "\"BINARY\"", "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", - "actualDataType" : "\"BINARY\"" + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 55, - "fragment" : "SELECT cast('1' as binary) / cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "cast('1' as binary) / cast('1' as binary)" } ] } @@ -1513,16 +1513,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS BOOLEAN))\"", "left" : "\"BINARY\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 54, - "fragment" : "SELECT cast('1' as binary) / cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast('1' as binary) / cast(1 as boolean)" } ] } @@ -1537,16 +1537,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"BINARY\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast('1' as binary) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('1' as binary) / cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -1561,16 +1561,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"BINARY\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 71, - "fragment" : "SELECT cast('1' as binary) / cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "cast('1' as binary) / cast('2017-12-11 09:30:00' as date)" } ] } @@ -1585,16 +1585,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS TINYINT))\"", "left" : "\"BOOLEAN\"", - "right" : "\"TINYINT\"" + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS TINYINT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 53, - "fragment" : "SELECT cast(1 as boolean) / cast(1 as tinyint) FROM t" + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(1 as boolean) / cast(1 as tinyint)" } ] } @@ -1609,16 +1609,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS SMALLINT))\"", "left" : "\"BOOLEAN\"", - "right" : "\"SMALLINT\"" + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS SMALLINT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 54, - "fragment" : "SELECT cast(1 as boolean) / cast(1 as smallint) FROM t" + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast(1 as boolean) / cast(1 as smallint)" } ] } @@ -1633,16 +1633,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS INT))\"", "left" : "\"BOOLEAN\"", - "right" : "\"INT\"" + "right" : "\"INT\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS INT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 49, - "fragment" : "SELECT cast(1 as boolean) / cast(1 as int) FROM t" + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "cast(1 as boolean) / cast(1 as int)" } ] } @@ -1657,16 +1657,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS BIGINT))\"", "left" : "\"BOOLEAN\"", - "right" : "\"BIGINT\"" + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS BIGINT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 52, - "fragment" : "SELECT cast(1 as boolean) / cast(1 as bigint) FROM t" + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as boolean) / cast(1 as bigint)" } ] } @@ -1681,16 +1681,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS FLOAT))\"", "left" : "\"BOOLEAN\"", - "right" : "\"FLOAT\"" + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS FLOAT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 51, - "fragment" : "SELECT cast(1 as boolean) / cast(1 as float) FROM t" + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "cast(1 as boolean) / cast(1 as float)" } ] } @@ -1705,16 +1705,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS DOUBLE))\"", "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS DOUBLE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 52, - "fragment" : "SELECT cast(1 as boolean) / cast(1 as double) FROM t" + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as boolean) / cast(1 as double)" } ] } @@ -1729,16 +1729,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BOOLEAN\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 60, - "fragment" : "SELECT cast(1 as boolean) / cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as boolean) / cast(1 as decimal(10, 0))" } ] } @@ -1753,16 +1753,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS STRING))\"", "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS STRING))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 52, - "fragment" : "SELECT cast(1 as boolean) / cast(1 as string) FROM t" + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as boolean) / cast(1 as string)" } ] } @@ -1777,16 +1777,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS BINARY))\"", "left" : "\"BOOLEAN\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 54, - "fragment" : "SELECT cast(1 as boolean) / cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast(1 as boolean) / cast('1' as binary)" } ] } @@ -1801,16 +1801,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_WRONG_TYPE", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS BOOLEAN))\"", + "actualDataType" : "\"BOOLEAN\"", "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", - "actualDataType" : "\"BOOLEAN\"" + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 53, - "fragment" : "SELECT cast(1 as boolean) / cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(1 as boolean) / cast(1 as boolean)" } ] } @@ -1825,16 +1825,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"BOOLEAN\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as boolean) / cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -1849,16 +1849,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"BOOLEAN\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 70, - "fragment" : "SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 63, + "fragment" : "cast(1 as boolean) / cast('2017-12-11 09:30:00' as date)" } ] } @@ -1873,16 +1873,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS TINYINT))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"TINYINT\"" + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS TINYINT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as tinyint) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as tinyint)" } ] } @@ -1897,16 +1897,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS SMALLINT))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"SMALLINT\"" + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS SMALLINT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as smallint) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as smallint)" } ] } @@ -1921,16 +1921,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS INT))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"INT\"" + "right" : "\"INT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS INT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 73, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as int) FROM t" + "startIndex" : 8, + "stopIndex" : 66, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as int)" } ] } @@ -1945,16 +1945,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS BIGINT))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"BIGINT\"" + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS BIGINT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 76, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as bigint) FROM t" + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as bigint)" } ] } @@ -1969,16 +1969,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS FLOAT))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"FLOAT\"" + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS FLOAT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 75, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as float) FROM t" + "startIndex" : 8, + "stopIndex" : 68, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as float)" } ] } @@ -1993,16 +1993,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DOUBLE))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DOUBLE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 76, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as double) FROM t" + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as double)" } ] } @@ -2017,16 +2017,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 84, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0))" } ] } @@ -2041,16 +2041,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS STRING))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS STRING))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 76, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as string) FROM t" + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as string)" } ] } @@ -2065,16 +2065,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS BINARY))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 78, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast('1' as binary)" } ] } @@ -2089,16 +2089,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS BOOLEAN))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as boolean)" } ] } @@ -2113,16 +2113,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_WRONG_TYPE", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", + "actualDataType" : "\"TIMESTAMP\"", "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", - "actualDataType" : "\"TIMESTAMP\"" + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 101, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 94, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -2137,16 +2137,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 94, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 87, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00' as date)" } ] } @@ -2161,16 +2161,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS TINYINT))\"", "left" : "\"DATE\"", - "right" : "\"TINYINT\"" + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS TINYINT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 70, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as tinyint) FROM t" + "startIndex" : 8, + "stopIndex" : 63, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as tinyint)" } ] } @@ -2185,16 +2185,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS SMALLINT))\"", "left" : "\"DATE\"", - "right" : "\"SMALLINT\"" + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS SMALLINT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 71, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as smallint) FROM t" + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as smallint)" } ] } @@ -2209,16 +2209,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS INT))\"", "left" : "\"DATE\"", - "right" : "\"INT\"" + "right" : "\"INT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS INT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 66, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as int) FROM t" + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as int)" } ] } @@ -2233,16 +2233,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS BIGINT))\"", "left" : "\"DATE\"", - "right" : "\"BIGINT\"" + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS BIGINT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 69, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as bigint) FROM t" + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as bigint)" } ] } @@ -2257,16 +2257,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS FLOAT))\"", "left" : "\"DATE\"", - "right" : "\"FLOAT\"" + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS FLOAT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 68, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as float) FROM t" + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as float)" } ] } @@ -2281,16 +2281,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS DOUBLE))\"", "left" : "\"DATE\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS DOUBLE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 69, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as double) FROM t" + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as double)" } ] } @@ -2305,16 +2305,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", - "right" : "\"DECIMAL(10,0)\"" + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(10,0)))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t" + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as decimal(10, 0))" } ] } @@ -2329,16 +2329,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS STRING))\"", "left" : "\"DATE\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS STRING))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 69, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as string) FROM t" + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as string)" } ] } @@ -2353,16 +2353,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS BINARY))\"", "left" : "\"DATE\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 71, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast('1' as binary)" } ] } @@ -2377,16 +2377,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS BOOLEAN))\"", "left" : "\"DATE\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 70, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 63, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as boolean)" } ] } @@ -2401,16 +2401,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DATE\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 94, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 87, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -2425,15 +2425,15 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_WRONG_TYPE", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(2017-12-11 09:30:00 AS DATE))\"", + "actualDataType" : "\"DATE\"", "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", - "actualDataType" : "\"DATE\"" + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 87, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00' as date)" } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out index 53a7269322a5a..edb8151ae794b 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 @@ -81,16 +81,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 + CAST(1 AS BINARY))\"", "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(1 + CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT '1' + cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "'1' + cast('1' as binary)" } ] } @@ -105,16 +105,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 + CAST(1 AS BOOLEAN))\"", "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(1 + CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT '1' + cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "'1' + cast(1 as boolean)" } ] } @@ -129,16 +129,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(1 + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT '1' + cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "'1' + cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -226,16 +226,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 - CAST(1 AS BINARY))\"", "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(1 - CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT '1' - cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "'1' - cast('1' as binary)" } ] } @@ -250,16 +250,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 - CAST(1 AS BOOLEAN))\"", "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(1 - CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT '1' - cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "'1' - cast(1 as boolean)" } ] } @@ -355,16 +355,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 * CAST(1 AS BINARY))\"", "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(1 * CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT '1' * cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "'1' * cast('1' as binary)" } ] } @@ -379,16 +379,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 * CAST(1 AS BOOLEAN))\"", "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(1 * CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT '1' * cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "'1' * cast(1 as boolean)" } ] } @@ -403,16 +403,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 * CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(1 * CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT '1' * cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "'1' * cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -427,16 +427,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 * CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DOUBLE\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(1 * CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT '1' * cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "'1' * cast('2017-12-11 09:30:00' as date)" } ] } @@ -515,16 +515,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 / CAST(1 AS BINARY))\"", "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(1 / CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT '1' / cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "'1' / cast('1' as binary)" } ] } @@ -539,16 +539,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 / CAST(1 AS BOOLEAN))\"", "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(1 / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT '1' / cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "'1' / cast(1 as boolean)" } ] } @@ -563,16 +563,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(1 / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT '1' / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "'1' / cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -587,16 +587,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DOUBLE\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(1 / CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT '1' / cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "'1' / cast('2017-12-11 09:30:00' as date)" } ] } @@ -675,16 +675,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 % CAST(1 AS BINARY))\"", "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"(1 % CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT '1' % cast('1' as binary) FROM t" + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "'1' % cast('1' as binary)" } ] } @@ -699,16 +699,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 % CAST(1 AS BOOLEAN))\"", "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(1 % CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT '1' % cast(1 as boolean) FROM t" + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "'1' % cast(1 as boolean)" } ] } @@ -723,16 +723,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(1 % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT '1' % cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "'1' % cast('2017-12-11 09:30:00.0' as timestamp)" } ] } @@ -747,16 +747,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(1 % CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DOUBLE\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"(1 % CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT '1' % cast('2017-12-11 09:30:00' as date) FROM t" + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "'1' % cast('2017-12-11 09:30:00' as date)" } ] } @@ -835,16 +835,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(1, CAST(1 AS BINARY))\"", "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"" + "right" : "\"BINARY\"", + "sqlExpr" : "\"pmod(1, CAST(1 AS BINARY))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 67, - "fragment" : "SELECT pmod('1', cast('1' as binary)) FROM t" + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "pmod('1', cast('1' as binary))" } ] } @@ -859,16 +859,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(1, CAST(1 AS BOOLEAN))\"", "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"" + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"pmod(1, CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 67, - "fragment" : "SELECT pmod('1', cast(1 as boolean)) FROM t" + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "pmod('1', cast(1 as boolean))" } ] } @@ -883,16 +883,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(1, CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"" + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"pmod(1, CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 67, - "fragment" : "SELECT pmod('1', cast('2017-12-11 09:30:00.0' as timestamp)) FROM t" + "startIndex" : 8, + "stopIndex" : 60, + "fragment" : "pmod('1', cast('2017-12-11 09:30:00.0' as timestamp))" } ] } @@ -907,16 +907,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(1, CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DOUBLE\"", - "right" : "\"DATE\"" + "right" : "\"DATE\"", + "sqlExpr" : "\"pmod(1, CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 67, - "fragment" : "SELECT pmod('1', cast('2017-12-11 09:30:00' as date)) FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "pmod('1', cast('2017-12-11 09:30:00' as date))" } ] } @@ -987,16 +987,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) + 1)\"", "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) + 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('1' as binary) + '1' FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) + '1'" } ] } @@ -1011,16 +1011,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BOOLEAN) + 1)\"", "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) + 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast(1 as boolean) + '1' FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as boolean) + '1'" } ] } @@ -1035,16 +1035,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + 1)\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) + '1' FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) + '1'" } ] } @@ -1124,16 +1124,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) - 1)\"", "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) - 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('1' as binary) - '1' FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) - '1'" } ] } @@ -1148,16 +1148,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BOOLEAN) - 1)\"", "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) - 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast(1 as boolean) - '1' FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as boolean) - '1'" } ] } @@ -1246,16 +1246,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) * 1)\"", "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) * 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('1' as binary) * '1' FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) * '1'" } ] } @@ -1270,16 +1270,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BOOLEAN) * 1)\"", "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) * 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast(1 as boolean) * '1' FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as boolean) * '1'" } ] } @@ -1294,16 +1294,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) * 1)\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) * 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) * '1' FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) * '1'" } ] } @@ -1318,16 +1318,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) * 1)\"", "left" : "\"DATE\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) * 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) * '1' FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00' as date) * '1'" } ] } @@ -1398,16 +1398,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) / 1)\"", "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('1' as binary) / '1' FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) / '1'" } ] } @@ -1422,16 +1422,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / 1)\"", "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast(1 as boolean) / '1' FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as boolean) / '1'" } ] } @@ -1446,16 +1446,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / 1)\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / '1' FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / '1'" } ] } @@ -1470,16 +1470,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / 1)\"", "left" : "\"DATE\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / '1' FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00' as date) / '1'" } ] } @@ -1550,16 +1550,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BINARY) % 1)\"", "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) % 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('1' as binary) % '1' FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) % '1'" } ] } @@ -1574,16 +1574,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(1 AS BOOLEAN) % 1)\"", "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) % 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast(1 as boolean) % '1' FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as boolean) % '1'" } ] } @@ -1598,16 +1598,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % 1)\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) % '1' FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) % '1'" } ] } @@ -1622,16 +1622,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % 1)\"", "left" : "\"DATE\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 62, - "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) % '1' FROM t" + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00' as date) % '1'" } ] } @@ -1702,16 +1702,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS BINARY), 1)\"", "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"pmod(CAST(1 AS BINARY), 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 67, - "fragment" : "SELECT pmod(cast('1' as binary), '1') FROM t" + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "pmod(cast('1' as binary), '1')" } ] } @@ -1726,16 +1726,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(1 AS BOOLEAN), 1)\"", "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"pmod(CAST(1 AS BOOLEAN), 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 67, - "fragment" : "SELECT pmod(cast(1 as boolean), '1') FROM t" + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "pmod(cast(1 as boolean), '1')" } ] } @@ -1750,16 +1750,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), 1)\"", "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 67, - "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), '1') FROM t" + "startIndex" : 8, + "stopIndex" : 60, + "fragment" : "pmod(cast('2017-12-11 09:30:00.0' as timestamp), '1')" } ] } @@ -1774,16 +1774,16 @@ org.apache.spark.sql.AnalysisException "errorClass" : "DATATYPE_MISMATCH", "errorSubClass" : "BINARY_OP_DIFF_TYPES", "messageParameters" : { - "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), 1)\"", "left" : "\"DATE\"", - "right" : "\"DOUBLE\"" + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), 1)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 67, - "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00' as date), '1') FROM t" + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "pmod(cast('2017-12-11 09:30:00' as date), '1')" } ] } 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 45e2cf187a31e..bd5e33ef9f7f9 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 @@ -140,8 +140,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'aa'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'aa'" } } @@ -164,8 +164,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'aa'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'aa'" } } @@ -188,8 +188,8 @@ org.apache.spark.SparkUpgradeException "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION", "errorSubClass" : "DATETIME_PATTERN_RECOGNITION", "messageParameters" : { - "pattern" : "'aa'", - "config" : "\"spark.sql.legacy.timeParserPolicy\"" + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "pattern" : "'aa'" } } 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 4cc2ca9fdecf0..ace8d483a985d 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 @@ -497,8 +497,8 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 100, - "fragment" : "select\n (select udf(max((select i.unique2 from tenk1 i where i.unique1 = o.unique1))))\nfrom tenk1 o" + "startIndex" : 75, + "stopIndex" : 83, + "fragment" : "o.unique1" } ] } 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 45f1a631b4597..de415c9971874 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 @@ -3284,9 +3284,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 91, - "fragment" : "select * from\n int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(y.f1)" + "startIndex" : 87, + "stopIndex" : 90, + "fragment" : "y.f1" } ] } @@ -3318,9 +3318,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 77, - "fragment" : "select udf(t1.uunique1) from\n tenk1 t1 join tenk2 t2 on t1.two = udf(t2.two)" + "startIndex" : 12, + "stopIndex" : 22, + "fragment" : "t1.uunique1" } ] } @@ -3343,9 +3343,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 82, - "fragment" : "select udf(udf(t2.uunique1)) from\n tenk1 t1 join tenk2 t2 on udf(t1.two) = t2.two" + "startIndex" : 16, + "stopIndex" : 26, + "fragment" : "t2.uunique1" } ] } @@ -3368,9 +3368,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 79, - "fragment" : "select udf(uunique1) from\n tenk1 t1 join tenk2 t2 on udf(t1.two) = udf(t2.two)" + "startIndex" : 12, + "stopIndex" : 19, + "fragment" : "uunique1" } ] } @@ -3582,9 +3582,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 68, - "fragment" : "select udf(udf(f1,g)) from int4_tbl a, (select udf(udf(f1)) as g) ss" + "startIndex" : 56, + "stopIndex" : 57, + "fragment" : "f1" } ] } @@ -3605,9 +3605,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 55, - "fragment" : "select udf(f1,g) from int4_tbl a, (select a.f1 as g) ss" + "startIndex" : 43, + "stopIndex" : 46, + "fragment" : "a.f1" } ] } @@ -3628,9 +3628,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 73, - "fragment" : "select udf(udf(f1,g)) from int4_tbl a cross join (select udf(f1) as g) ss" + "startIndex" : 62, + "stopIndex" : 63, + "fragment" : "f1" } ] } @@ -3651,9 +3651,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 75, - "fragment" : "select udf(f1,g) from int4_tbl a cross join (select udf(udf(a.f1)) as g) ss" + "startIndex" : 61, + "stopIndex" : 64, + "fragment" : "a.f1" } ] } 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 d6c39b73df994..4bda849709df5 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 @@ -160,9 +160,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 50, - "fragment" : "SELECT 1 AS one FROM test_having HAVING udf(a) > 1" + "startIndex" : 45, + "stopIndex" : 45, + "fragment" : "a" } ] } 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 804f7287437b2..f448206594897 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 @@ -133,9 +133,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 63, - "stopIndex" : 77, - "fragment" : "ORDER BY udf(b)" + "startIndex" : 76, + "stopIndex" : 76, + "fragment" : "b" } ] } @@ -367,9 +367,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 68, - "stopIndex" : 82, - "fragment" : "ORDER BY udf(b)" + "startIndex" : 81, + "stopIndex" : 81, + "fragment" : "b" } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out index 6e5f9c4ff9946..969ad69c27996 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out @@ -48,14 +48,7 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "expression" : "\"a\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 49, - "stopIndex" : 58, - "fragment" : "GROUP BY b" - } ] + } } @@ -125,14 +118,7 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "expression" : "\"a\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 48, - "stopIndex" : 61, - "fragment" : "GROUP BY a + 1" - } ] + } } @@ -213,14 +199,7 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "expression" : "\"k\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 69, - "stopIndex" : 83, - "fragment" : "GROUP BY udf(a)" - } ] + } } @@ -249,9 +228,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 49, + "startIndex" : 58, "stopIndex" : 58, - "fragment" : "GROUP BY k" + "fragment" : "k" } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out index dad37991f78af..3390b139c5b8c 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 @@ -240,9 +240,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 62, "stopIndex" : 118, - "fragment" : "SELECT * FROM (\n SELECT course, earnings FROM courseSales\n)\nPIVOT (\n udf(sum(earnings))\n FOR year IN (2012, 2013)\n)" + "fragment" : "PIVOT (\n udf(sum(earnings))\n FOR year IN (2012, 2013)\n)" } ] } @@ -329,9 +329,9 @@ org.apache.spark.sql.AnalysisException "errorClass" : "PIVOT_VALUE_DATA_TYPE_MISMATCH", "sqlState" : "42000", "messageParameters" : { + "pivotType" : "struct", "value" : "dotNET", - "valueType" : "string", - "pivotType" : "struct" + "valueType" : "string" } } @@ -357,9 +357,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 80, - "fragment" : "SELECT * FROM courseSales\nPIVOT (\n udf(sum(earnings))\n FOR year IN (s, 2013)\n)" + "startIndex" : 71, + "stopIndex" : 71, + "fragment" : "s" } ] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index efe7cb8576496..59382d220f5b8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -2537,8 +2537,13 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException](df.selectExpr("transform(a, x -> x)")), errorClass = "UNRESOLVED_COLUMN", - errorSubClass = Some("WITH_SUGGESTION"), - parameters = Map("objectName" -> "`a`", "proposal" -> "`i`, `s`")) + errorSubClass = "WITH_SUGGESTION", + sqlState = None, + parameters = Map("objectName" -> "`a`", "proposal" -> "`i`, `s`"), + context = ExpectedContext( + fragment = "a", + start = 10, + stop = 10)) } test("map_filter") { @@ -2610,8 +2615,14 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException](df.selectExpr("map_filter(a, (k, v) -> k > v)")), errorClass = "UNRESOLVED_COLUMN", - errorSubClass = Some("WITH_SUGGESTION"), - parameters = Map("objectName" -> "`a`", "proposal" -> "`i`, `s`")) + errorSubClass = "WITH_SUGGESTION", + sqlState = None, + parameters = Map("objectName" -> "`a`", "proposal" -> "`i`, `s`"), + context = ExpectedContext( + fragment = "a", + start = 11, + stop = 11) + ) } test("filter function - array for primitive type not containing null") { @@ -2771,8 +2782,13 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException](df.selectExpr("filter(a, x -> x)")), errorClass = "UNRESOLVED_COLUMN", - errorSubClass = Some("WITH_SUGGESTION"), - parameters = Map("objectName" -> "`a`", "proposal" -> "`i`, `s`")) + errorSubClass = "WITH_SUGGESTION", + sqlState = None, + parameters = Map("objectName" -> "`a`", "proposal" -> "`i`, `s`"), + context = ExpectedContext( + fragment = "a", + start = 7, + stop = 7)) } test("exists function - array for primitive type not containing null") { @@ -2905,8 +2921,13 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException](df.selectExpr("exists(a, x -> x)")), errorClass = "UNRESOLVED_COLUMN", - errorSubClass = Some("WITH_SUGGESTION"), - parameters = Map("objectName" -> "`a`", "proposal" -> "`i`, `s`")) + errorSubClass = "WITH_SUGGESTION", + sqlState = None, + parameters = Map("objectName" -> "`a`", "proposal" -> "`i`, `s`"), + context = ExpectedContext( + fragment = "a", + start = 7, + stop = 7)) } test("forall function - array for primitive type not containing null") { @@ -3053,8 +3074,13 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException](df.selectExpr("forall(a, x -> x)")), errorClass = "UNRESOLVED_COLUMN", - errorSubClass = Some("WITH_SUGGESTION"), - parameters = Map("objectName" -> "`a`", "proposal" -> "`i`, `s`")) + errorSubClass = "WITH_SUGGESTION", + sqlState = None, + parameters = Map("objectName" -> "`a`", "proposal" -> "`i`, `s`"), + context = ExpectedContext( + fragment = "a", + start = 7, + stop = 7)) checkError( exception = @@ -3239,8 +3265,13 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException](df.selectExpr("aggregate(a, 0, (acc, x) -> x)")), errorClass = "UNRESOLVED_COLUMN", - errorSubClass = Some("WITH_SUGGESTION"), - parameters = Map("objectName" -> "`a`", "proposal" -> "`i`, `s`")) + errorSubClass = "WITH_SUGGESTION", + sqlState = None, + parameters = Map("objectName" -> "`a`", "proposal" -> "`i`, `s`"), + context = ExpectedContext( + fragment = "a", + start = 10, + stop = 10)) } test("map_zip_with function - map of primitive types") { @@ -3795,8 +3826,14 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { exception = intercept[AnalysisException](df.selectExpr("zip_with(a1, a, (acc, x) -> x)")), errorClass = "UNRESOLVED_COLUMN", - errorSubClass = Some("WITH_SUGGESTION"), - parameters = Map("objectName" -> "`a`", "proposal" -> "`a1`, `a2`, `i`")) + errorSubClass = "WITH_SUGGESTION", + sqlState = None, + parameters = Map("objectName" -> "`a`", "proposal" -> "`a1`, `a2`, `i`"), + context = ExpectedContext( + fragment = "a", + start = 13, + stop = 13) + ) } private def assertValuesDoNotChangeAfterCoalesceOrUnion(v: Column): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 9dca09fa2e9d7..1dcf0b6e4a99b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -483,7 +483,7 @@ class DataFrameSuite extends QueryTest testData.select("key").coalesce(1).select("key"), testData.select("key").collect().toSeq) - assert(spark.emptyDataFrame.coalesce(1).rdd.partitions.size === 0) + assert(spark.emptyDataFrame.coalesce(1).rdd.partitions.size === 1) } test("convert $\"attribute name\" into unresolved attribute") { @@ -2784,11 +2784,13 @@ class DataFrameSuite extends QueryTest implicit val valueEncoder = RowEncoder(df.schema) - val err = intercept[AnalysisException] { - df.groupBy($"d", $"b").as[GroupByKey, Row] - } - assert(err.getErrorClass == "UNRESOLVED_COLUMN") - assert(err.messageParameters.head == "`d`") + checkError( + exception = intercept[AnalysisException] { + df.groupBy($"d", $"b").as[GroupByKey, Row] + }, + errorClass = "UNRESOLVED_COLUMN", + errorSubClass = Some("WITH_SUGGESTION"), + parameters = Map("objectName" -> "`d`", "proposal" -> "`a`, `b`, `c`")) } test("emptyDataFrame should be foldable") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index d7ea766b21b63..5be6d53f6e10c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -327,20 +327,30 @@ class DatasetSuite extends QueryTest ds.select(expr("`(_1)?+.+`").as[Int]) }, errorClass = "UNRESOLVED_COLUMN", - errorSubClass = Some("WITH_SUGGESTION"), + errorSubClass = "WITH_SUGGESTION", + sqlState = None, parameters = Map( "objectName" -> "`(_1)?+.+`", - "proposal" -> "`_1`, `_2`")) + "proposal" -> "`_1`, `_2`"), + context = ExpectedContext( + fragment = "`(_1)?+.+`", + start = 0, + stop = 9)) checkError( exception = intercept[AnalysisException] { ds.select(expr("`(_1|_2)`").as[Int]) }, errorClass = "UNRESOLVED_COLUMN", - errorSubClass = Some("WITH_SUGGESTION"), + errorSubClass = "WITH_SUGGESTION", + sqlState = None, parameters = Map( "objectName" -> "`(_1|_2)`", - "proposal" -> "`_1`, `_2`")) + "proposal" -> "`_1`, `_2`"), + context = ExpectedContext( + fragment = "`(_1|_2)`", + start = 0, + stop = 8)) var e = intercept[AnalysisException] { ds.select(ds("`(_1)?+.+`")) 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 96825c8003f4a..215b005319e29 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 @@ -2676,9 +2676,10 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark "left" -> "\"STRUCT\"", "right" -> "\"STRUCT\""), context = ExpectedContext( - fragment = query, - start = 0, - stop = 29)) + fragment = "c = C", + start = 25, + stop = 29 + )) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 6116b1aa6794b..2c2c3b86317e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -899,7 +899,9 @@ class SubquerySuite extends QueryTest "objectName" -> "`a`", "proposal" -> "`t`.`i`, `t`.`j`"), context = ExpectedContext( - fragment = query, start = 0, stop = 42)) + fragment = "a", + start = 37, + stop = 37)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 3a77b12a1ecfa..11f4fe0649be4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -122,7 +122,9 @@ class DataSourceV2SQLSuiteV1Filter extends DataSourceV2SQLSuite with AlterTableT s"DESCRIBE $t invalid_col", "UNRESOLVED_COLUMN", "WITH_SUGGESTION", - Array("`invalid_col`", "`testcat`.`tbl`.`id`, `testcat`.`tbl`.`data`")) + Map( + "objectName" -> "`invalid_col`", + "proposal" -> "`testcat`.`tbl`.`id`, `testcat`.`tbl`.`data`")) } } @@ -997,8 +999,9 @@ class DataSourceV2SQLSuiteV1Filter extends DataSourceV2SQLSuite with AlterTableT s"SELECT ns1.ns2.ns3.tbl.id from $t", "UNRESOLVED_COLUMN", "WITH_SUGGESTION", - Array("`ns1`.`ns2`.`ns3`.`tbl`.`id`", - "`testcat`.`ns1`.`ns2`.`tbl`.`id`, `testcat`.`ns1`.`ns2`.`tbl`.`point`")) + Map( + "objectName" -> "`ns1`.`ns2`.`ns3`.`tbl`.`id`", + "proposal" -> "`testcat`.`ns1`.`ns2`.`tbl`.`id`, `testcat`.`ns1`.`ns2`.`tbl`.`point`")) } } @@ -1577,19 +1580,19 @@ class DataSourceV2SQLSuiteV1Filter extends DataSourceV2SQLSuite with AlterTableT s"UPDATE $t SET dummy='abc'", "UNRESOLVED_COLUMN", "WITH_SUGGESTION", - Array( - "`dummy`", - "`testcat`.`ns1`.`ns2`.`tbl`.`p`, `testcat`.`ns1`.`ns2`.`tbl`.`id`, " + - "`testcat`.`ns1`.`ns2`.`tbl`.`age`, `testcat`.`ns1`.`ns2`.`tbl`.`name`")) + Map( + "objectName" -> "`dummy`", + "proposal" -> ("`testcat`.`ns1`.`ns2`.`tbl`.`p`, `testcat`.`ns1`.`ns2`.`tbl`.`id`, " + + "`testcat`.`ns1`.`ns2`.`tbl`.`age`, `testcat`.`ns1`.`ns2`.`tbl`.`name`"))) assertAnalysisErrorClass( s"UPDATE $t SET name='abc' WHERE dummy=1", "UNRESOLVED_COLUMN", "WITH_SUGGESTION", - Array( - "`dummy`", - "`testcat`.`ns1`.`ns2`.`tbl`.`p`, " + + Map( + "objectName" -> "`dummy`", + "proposal" -> ("`testcat`.`ns1`.`ns2`.`tbl`.`p`, " + "`testcat`.`ns1`.`ns2`.`tbl`.`id`, " + - "`testcat`.`ns1`.`ns2`.`tbl`.`age`, `testcat`.`ns1`.`ns2`.`tbl`.`name`")) + "`testcat`.`ns1`.`ns2`.`tbl`.`age`, `testcat`.`ns1`.`ns2`.`tbl`.`name`"))) // UPDATE is not implemented yet. val e = intercept[UnsupportedOperationException] { @@ -2428,7 +2431,7 @@ class DataSourceV2SQLSuiteV1Filter extends DataSourceV2SQLSuite with AlterTableT sqlStatement: String, expectedErrorClass: String, expectedErrorSubClass: String, - expectedErrorMessageParameters: Array[String]): Unit = { + expectedErrorMessageParameters: Map[String, String]): Unit = { val ex = intercept[AnalysisException] { sql(sqlStatement) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala index f07451b4dd560..00c774e2d1bee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala @@ -413,11 +413,14 @@ class QueryCompilationErrorsSuite errorClass = "UNRESOLVED_MAP_KEY", errorSubClass = "WITH_SUGGESTION", sqlState = None, - parameters = Map("columnName" -> "`a`", + parameters = Map("objectName" -> "`a`", "proposal" -> "`__auto_generated_subquery_name`.`m`, `__auto_generated_subquery_name`.`aa`"), context = ExpectedContext( - fragment = query, start = 0, stop = 55)) + fragment = "a", + start = 9, + stop = 9) + ) } test("UNRESOLVED_COLUMN: SELECT distinct does not work correctly " + @@ -451,7 +454,9 @@ class QueryCompilationErrorsSuite "proposal" -> "`a`, `b`" ), context = ExpectedContext( - fragment = "order by struct.a, struct.b", start = 171, stop = 197) + fragment = "struct.a", + start = 180, + stop = 187) ) } @@ -470,7 +475,9 @@ class QueryCompilationErrorsSuite "objectName" -> "`v`.`i`", "proposal" -> "`__auto_generated_subquery_name`.`i`"), context = ExpectedContext( - fragment = query, start = 0, stop = 32)) + fragment = "v.i", + start = 7, + stop = 9)) checkAnswer(sql("SELECT __auto_generated_subquery_name.i from (SELECT i FROM v)"), Row(1)) } @@ -574,7 +581,8 @@ class QueryCompilationErrorsSuite exception = e1, errorClass = "UNSUPPORTED_DESERIALIZER", errorSubClass = Some("FIELD_NUMBER_MISMATCH"), - parameters = Map("schema" -> "\"STRUCT\"", + parameters = Map( + "schema" -> "\"STRUCT\"", "ordinal" -> "3")) val e2 = intercept[AnalysisException] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index e55ec53f927cc..0430d314a1ba3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -650,7 +650,7 @@ class QueryExecutionErrorsSuite parameters = Map( "message" -> "integer overflow", "alternative" -> "", - "config" -> SQLConf.ANSI_ENABLED.key)) + "config" -> s""""${SQLConf.ANSI_ENABLED.key}"""")) } test("CAST_OVERFLOW: from long to ANSI intervals") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala index be438a37a6b70..cd46cc3b659bb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala @@ -183,7 +183,8 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession { exception = parseException("SELECT * FROM db.func()"), errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - parameters = Map("inputString" -> "table valued function cannot specify database name "), + parameters = Map( + "inputString" -> "table valued function cannot specify database name: `db`.`func`"), context = ExpectedContext( fragment = "db.func()", start = 14, @@ -193,7 +194,8 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession { exception = parseException("SELECT * FROM ns.db.func()"), errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - parameters = Map("inputString" -> "table valued function cannot specify database name "), + parameters = Map( + "inputString" -> "table valued function cannot specify database name: `ns`.`db`.`func`"), context = ExpectedContext( fragment = "ns.db.func()", start = 14, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index ddf500b3f93bd..8ce575d12a45e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -2643,6 +2643,49 @@ class AdaptiveQueryExecSuite assert(findTopLevelBroadcastNestedLoopJoin(adaptivePlan).size == 1) } } + + test("SPARK-39915: Dataset.repartition(N) may not create N partitions") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "6") { + // partitioning: HashPartitioning + // shuffleOrigin: REPARTITION_BY_NUM + assert(spark.range(0).repartition(5, $"id").rdd.getNumPartitions == 5) + // shuffleOrigin: REPARTITION_BY_COL + // The minimum partition number after AQE coalesce is 1 + assert(spark.range(0).repartition($"id").rdd.getNumPartitions == 1) + // through project + assert(spark.range(0).selectExpr("id % 3 as c1", "id % 7 as c2") + .repartition(5, $"c1").select($"c2").rdd.getNumPartitions == 5) + + // partitioning: RangePartitioning + // shuffleOrigin: REPARTITION_BY_NUM + // The minimum partition number of RangePartitioner is 1 + assert(spark.range(0).repartitionByRange(5, $"id").rdd.getNumPartitions == 1) + // shuffleOrigin: REPARTITION_BY_COL + assert(spark.range(0).repartitionByRange($"id").rdd.getNumPartitions == 1) + + // partitioning: RoundRobinPartitioning + // shuffleOrigin: REPARTITION_BY_NUM + assert(spark.range(0).repartition(5).rdd.getNumPartitions == 5) + // shuffleOrigin: REBALANCE_PARTITIONS_BY_NONE + assert(spark.range(0).repartition().rdd.getNumPartitions == 0) + // through project + assert(spark.range(0).selectExpr("id % 3 as c1", "id % 7 as c2") + .repartition(5).select($"c2").rdd.getNumPartitions == 5) + + // partitioning: SinglePartition + assert(spark.range(0).repartition(1).rdd.getNumPartitions == 1) + } + } + + test("SPARK-39915: Ensure the output partitioning is user-specified") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "3", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df1 = spark.range(1).selectExpr("id as c1") + val df2 = spark.range(1).selectExpr("id as c2") + val df = df1.join(df2, col("c1") === col("c2")).repartition(3, col("c1")) + assert(df.rdd.getNumPartitions == 3) + } + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala index d6b2c8f48f5e5..091993c63499a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala @@ -112,7 +112,10 @@ class DescribeTableSuite extends command.DescribeTableSuiteBase "objectName" -> "`key1`", "proposal" -> "`test_catalog`.`ns`.`tbl`.`key`, `test_catalog`.`ns`.`tbl`.`col`"), context = ExpectedContext( - fragment = query, start = 0, stop = 28)) + fragment = query, + start = 0, + stop = query.length -1) + ) } } @@ -141,7 +144,9 @@ class DescribeTableSuite extends command.DescribeTableSuiteBase "objectName" -> "`KEY`", "proposal" -> "`test_catalog`.`ns`.`tbl`.`key`"), context = ExpectedContext( - fragment = query, start = 0, stop = 27)) + fragment = query, + start = 0, + stop = query.length - 1)) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 5c97821f11ecd..1852e13181674 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -36,7 +36,7 @@ import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.GzipCodec import org.apache.logging.log4j.Level -import org.apache.spark.{SparkConf, SparkException, SparkUpgradeException, TestUtils} +import org.apache.spark.{SparkConf, SparkException, SparkIllegalArgumentException, SparkUpgradeException, TestUtils} import org.apache.spark.sql.{AnalysisException, Column, DataFrame, Encoders, QueryTest, Row} import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils} import org.apache.spark.sql.execution.datasources.CommonFileDataSourceSuite @@ -2653,11 +2653,13 @@ abstract class CSVSuite .option("header", true) .csv(path.getCanonicalPath) checkAnswer(readback, Seq(Row(2, 3), Row(0, 1))) - val ex = intercept[AnalysisException] { - readback.filter($"AAA" === 2 && $"bbb" === 3).collect() - } - assert(ex.getErrorClass == "UNRESOLVED_COLUMN") - assert(ex.messageParameters.head == "`AAA`") + checkError( + exception = intercept[AnalysisException] { + readback.filter($"AAA" === 2 && $"bbb" === 3).collect() + }, + errorClass = "UNRESOLVED_COLUMN", + errorSubClass = Some("WITH_SUGGESTION"), + parameters = Map("objectName" -> "`AAA`", "proposal" -> "`BBB`, `aaa`")) } } } @@ -2805,13 +2807,11 @@ abstract class CSVSuite // Error should be thrown when attempting to prefersDate with Legacy parser if (SQLConf.get.legacyTimeParserPolicy == LegacyBehaviorPolicy.LEGACY) { - val msg = intercept[IllegalArgumentException] { - spark.read - .format("csv") - .options(options1) - .load(testFile(dateInferSchemaFile)) - }.getMessage - assert(msg.contains("CANNOT_INFER_DATE")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + spark.read.format("csv").options(options1).load(testFile(dateInferSchemaFile)) + }, + errorClass = "CANNOT_INFER_DATE") } else { // 1. Specify date format and timestamp format // 2. Date inference should work with default date format when dateFormat is not provided @@ -2859,10 +2859,11 @@ abstract class CSVSuite .csv(path.getAbsolutePath) if (SQLConf.get.legacyTimeParserPolicy == LegacyBehaviorPolicy.LEGACY) { - val msg = intercept[IllegalArgumentException] { - output.collect() - }.getMessage - assert(msg.contains("CANNOT_INFER_DATE")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + output.collect() + }, + errorClass = "CANNOT_INFER_DATE") } else { checkAnswer( output, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index f0801ae313e8c..a9e1d3a751ed5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -3041,11 +3041,13 @@ abstract class JsonSuite val readback = spark.read.schema("aaa integer, BBB integer") .json(path.getCanonicalPath) checkAnswer(readback, Seq(Row(null, null), Row(0, 1))) - val ex = intercept[AnalysisException] { - readback.filter($"AAA" === 0 && $"bbb" === 1).collect() - } - assert(ex.getErrorClass == "UNRESOLVED_COLUMN") - assert(ex.messageParameters.head == "`AAA`") + checkError( + exception = intercept[AnalysisException] { + readback.filter($"AAA" === 0 && $"bbb" === 1).collect() + }, + errorClass = "UNRESOLVED_COLUMN", + errorSubClass = Some("WITH_SUGGESTION"), + parameters = Map("objectName" -> "`AAA`", "proposal" -> "`BBB`, `aaa`")) // Schema inferring val readback2 = spark.read.json(path.getCanonicalPath) checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index fa3241fe59bf3..c3b3ba6a17477 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -2092,19 +2092,30 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { test("SPARK-33294: Add query resolved check before analyze InsertIntoDir") { withTempPath { path => - val ex = intercept[AnalysisException] { - sql( - s""" - |INSERT OVERWRITE DIRECTORY '${path.getAbsolutePath}' USING PARQUET - |SELECT * FROM ( - | SELECT c3 FROM ( - | SELECT c1, c2 from values(1,2) t(c1, c2) - | ) - |) - """.stripMargin) - } - assert(ex.getErrorClass == "UNRESOLVED_COLUMN") - assert(ex.messageParameters.head == "`c3`") + val insert = s"INSERT OVERWRITE DIRECTORY '${path.getAbsolutePath}' USING PARQUET" + checkError( + exception = intercept[AnalysisException] { + sql( + s""" + |$insert + |SELECT * FROM ( + | SELECT c3 FROM ( + | SELECT c1, c2 from values(1,2) t(c1, c2) + | ) + |) + """.stripMargin) + }, + errorClass = "UNRESOLVED_COLUMN", + errorSubClass = "WITH_SUGGESTION", + sqlState = "42000", + parameters = Map( + "objectName" -> "`c3`", + "proposal" -> + "`__auto_generated_subquery_name`.`c1`, `__auto_generated_subquery_name`.`c2`"), + context = ExpectedContext( + fragment = "c3", + start = insert.length + 26, + stop = insert.length + 27)) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala index 50c08d8807447..756abe297a61b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala @@ -135,7 +135,9 @@ class HiveParquetSuite extends QueryTest "proposal" -> ("`__auto_generated_subquery_name`.`c1`, " + "`__auto_generated_subquery_name`.`c2`")), context = ExpectedContext( - fragment = query.trim, start = 1, stop = 118) + fragment = "c3", + start = 61, + stop = 62) ) } }