diff --git a/core/src/main/resources/error/README.md b/core/src/main/resources/error/README.md index f58eb6d9296ef..23e62cd25fb42 100644 --- a/core/src/main/resources/error/README.md +++ b/core/src/main/resources/error/README.md @@ -14,7 +14,7 @@ and message parameters rather than an arbitrary error message. 4. Check if the exception type already extends `SparkThrowable`. If true, skip to step 6. 5. Mix `SparkThrowable` into the exception. -6. Throw the exception with the error class and message parameters. +6. Throw the exception with the error class and message parameters. If the same exception is thrown in several places, create an util function in a central place such as `QueryCompilationErrors.scala` to instantiate the exception. ### Before @@ -66,6 +66,8 @@ To access error fields, catch exceptions that extend `org.apache.spark.SparkThro Error classes are a succinct, human-readable representation of the error category. +An uncategorized errors can be assigned to a legacy error class with the prefix `_LEGACY_ERROR_TEMP_` and an unused sequential number, for instance `_LEGACY_ERROR_TEMP_0053`. + #### Invariants - Unique diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 6fdf2b577e046..823780264cb29 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -819,5 +819,325 @@ "Writing job aborted" ], "sqlState" : "40000" + }, + "_LEGACY_ERROR_TEMP_0001" : { + "message" : [ + "Invalid InsertIntoContext" + ] + }, + "_LEGACY_ERROR_TEMP_0002" : { + "message" : [ + "INSERT OVERWRITE DIRECTORY is not supported" + ] + }, + "_LEGACY_ERROR_TEMP_0003" : { + "message" : [ + "Columns aliases are not allowed in ." + ] + }, + "_LEGACY_ERROR_TEMP_0004" : { + "message" : [ + "Empty source for merge: you should specify a source table/subquery in merge." + ] + }, + "_LEGACY_ERROR_TEMP_0005" : { + "message" : [ + "Unrecognized matched action: ." + ] + }, + "_LEGACY_ERROR_TEMP_0006" : { + "message" : [ + "The number of inserted values cannot match the fields." + ] + }, + "_LEGACY_ERROR_TEMP_0007" : { + "message" : [ + "Unrecognized not matched action: ." + ] + }, + "_LEGACY_ERROR_TEMP_0008" : { + "message" : [ + "There must be at least one WHEN clause in a MERGE statement." + ] + }, + "_LEGACY_ERROR_TEMP_0009" : { + "message" : [ + "When there are more than one MATCHED clauses in a MERGE statement, only the last MATCHED clause can omit the condition." + ] + }, + "_LEGACY_ERROR_TEMP_0010" : { + "message" : [ + "When there are more than one NOT MATCHED clauses in a MERGE statement, only the last NOT MATCHED clause can omit the condition." + ] + }, + "_LEGACY_ERROR_TEMP_0011" : { + "message" : [ + "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported." + ] + }, + "_LEGACY_ERROR_TEMP_0012" : { + "message" : [ + "DISTRIBUTE BY is not supported." + ] + }, + "_LEGACY_ERROR_TEMP_0013" : { + "message" : [ + "LATERAL cannot be used together with PIVOT in FROM clause." + ] + }, + "_LEGACY_ERROR_TEMP_0014" : { + "message" : [ + "TABLESAMPLE does not accept empty inputs." + ] + }, + "_LEGACY_ERROR_TEMP_0015" : { + "message" : [ + "TABLESAMPLE() is not supported." + ] + }, + "_LEGACY_ERROR_TEMP_0016" : { + "message" : [ + " is not a valid byte length literal, expected syntax: DIGIT+ ('B' | 'K' | 'M' | 'G')." + ] + }, + "_LEGACY_ERROR_TEMP_0017" : { + "message" : [ + "Invalid escape string. Escape string must contain only one character." + ] + }, + "_LEGACY_ERROR_TEMP_0018" : { + "message" : [ + "Function trim doesn't support with type . Please use BOTH, LEADING or TRAILING as trim type." + ] + }, + "_LEGACY_ERROR_TEMP_0019" : { + "message" : [ + "Cannot parse the value: ." + ] + }, + "_LEGACY_ERROR_TEMP_0020" : { + "message" : [ + "Cannot parse the INTERVAL value: ." + ] + }, + "_LEGACY_ERROR_TEMP_0021" : { + "message" : [ + "Literals of type '' are currently not supported." + ] + }, + "_LEGACY_ERROR_TEMP_0022" : { + "message" : [ + "." + ] + }, + "_LEGACY_ERROR_TEMP_0023" : { + "message" : [ + "Numeric literal does not fit in range [, ] for type ." + ] + }, + "_LEGACY_ERROR_TEMP_0024" : { + "message" : [ + "Can only have a single from-to unit in the interval literal syntax." + ] + }, + "_LEGACY_ERROR_TEMP_0025" : { + "message" : [ + "At least one time unit should be given for interval literal." + ] + }, + "_LEGACY_ERROR_TEMP_0026" : { + "message" : [ + "Can only use numbers in the interval value part for multiple unit value pairs interval form, but got invalid value: ." + ] + }, + "_LEGACY_ERROR_TEMP_0027" : { + "message" : [ + "The value of from-to unit must be a string." + ] + }, + "_LEGACY_ERROR_TEMP_0028" : { + "message" : [ + "Intervals FROM TO are not supported." + ] + }, + "_LEGACY_ERROR_TEMP_0029" : { + "message" : [ + "Cannot mix year-month and day-time fields: ." + ] + }, + "_LEGACY_ERROR_TEMP_0030" : { + "message" : [ + "DataType is not supported." + ] + }, + "_LEGACY_ERROR_TEMP_0031" : { + "message" : [ + "Invalid number of buckets: ." + ] + }, + "_LEGACY_ERROR_TEMP_0032" : { + "message" : [ + "Duplicated table paths found: '' and ''. LOCATION and the case insensitive key 'path' in OPTIONS are all used to indicate the custom table path, you can only specify one of them." + ] + }, + "_LEGACY_ERROR_TEMP_0033" : { + "message" : [ + "Expected either STORED AS or STORED BY, not both." + ] + }, + "_LEGACY_ERROR_TEMP_0034" : { + "message" : [ + " is not supported in Hive-style ." + ] + }, + "_LEGACY_ERROR_TEMP_0035" : { + "message" : [ + "Operation not allowed: ." + ] + }, + "_LEGACY_ERROR_TEMP_0036" : { + "message" : [ + "Expected `NOSCAN` instead of ``." + ] + }, + "_LEGACY_ERROR_TEMP_0037" : { + "message" : [ + "It is not allowed to add catalog/namespace prefix to the table name in CACHE TABLE AS SELECT." + ] + }, + "_LEGACY_ERROR_TEMP_0038" : { + "message" : [ + "CTE definition can't have duplicate names: ." + ] + }, + "_LEGACY_ERROR_TEMP_0039" : { + "message" : [ + "Unsupported SQL statement." + ] + }, + "_LEGACY_ERROR_TEMP_0040" : { + "message" : [ + "Possibly unquoted identifier detected. Please consider quoting it with back-quotes as ``." + ] + }, + "_LEGACY_ERROR_TEMP_0041" : { + "message" : [ + "Found duplicate clauses: ." + ] + }, + "_LEGACY_ERROR_TEMP_0042" : { + "message" : [ + "Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include special characters in key, or include semicolon in value, please use quotes, e.g., SET `key`=`value`." + ] + }, + "_LEGACY_ERROR_TEMP_0043" : { + "message" : [ + "Expected format is 'RESET' or 'RESET key'. If you want to include special characters in key, please use quotes, e.g., RESET `key`." + ] + }, + "_LEGACY_ERROR_TEMP_0044" : { + "message" : [ + "The interval value must be in the range of [-18, +18] hours with second precision." + ] + }, + "_LEGACY_ERROR_TEMP_0045" : { + "message" : [ + "Invalid time zone displacement value." + ] + }, + "_LEGACY_ERROR_TEMP_0046" : { + "message" : [ + "CREATE TEMPORARY TABLE without a provider is not allowed." + ] + }, + "_LEGACY_ERROR_TEMP_0047" : { + "message" : [ + "'ROW FORMAT' must be used with 'STORED AS'." + ] + }, + "_LEGACY_ERROR_TEMP_0048" : { + "message" : [ + "Unsupported operation: Used defined record reader/writer classes." + ] + }, + "_LEGACY_ERROR_TEMP_0049" : { + "message" : [ + "Directory path and 'path' in OPTIONS should be specified one, but not both." + ] + }, + "_LEGACY_ERROR_TEMP_0050" : { + "message" : [ + "LOCAL is supported only with file: scheme." + ] + }, + "_LEGACY_ERROR_TEMP_0051" : { + "message" : [ + "Empty set in grouping sets is not supported." + ] + }, + "_LEGACY_ERROR_TEMP_0052" : { + "message" : [ + "CREATE VIEW with both IF NOT EXISTS and REPLACE is not allowed." + ] + }, + "_LEGACY_ERROR_TEMP_0053" : { + "message" : [ + "It is not allowed to define a TEMPORARY view with IF NOT EXISTS." + ] + }, + "_LEGACY_ERROR_TEMP_0054" : { + "message" : [ + "It is not allowed to add database prefix `` for the TEMPORARY view name." + ] + }, + "_LEGACY_ERROR_TEMP_0055" : { + "message" : [ + "Unclosed bracketed comment" + ] + }, + "_LEGACY_ERROR_TEMP_0056" : { + "message" : [ + "Invalid time travel spec: ." + ] + }, + "_LEGACY_ERROR_TEMP_0057" : { + "message" : [ + "Support for DEFAULT column values is not implemented yet." + ] + }, + "_LEGACY_ERROR_TEMP_0058" : { + "message" : [ + "Support for DEFAULT column values is not allowed." + ] + }, + "_LEGACY_ERROR_TEMP_0059" : { + "message" : [ + "References to DEFAULT column values are not allowed within the PARTITION clause." + ] + }, + "_LEGACY_ERROR_TEMP_0060" : { + "message" : [ + "." + ] + }, + "_LEGACY_ERROR_TEMP_0061" : { + "message" : [ + "." + ] + }, + "_LEGACY_ERROR_TEMP_0062" : { + "message" : [ + "." + ] + }, + "_LEGACY_ERROR_TEMP_0063" : { + "message" : [ + "." + ] + }, + "_LEGACY_ERROR_TEMP_0064" : { + "message" : [ + "." + ] } } diff --git a/core/src/main/scala/org/apache/spark/SparkThrowableHelper.scala b/core/src/main/scala/org/apache/spark/SparkThrowableHelper.scala index 86337205d31c8..1a50c1acd8df1 100644 --- a/core/src/main/scala/org/apache/spark/SparkThrowableHelper.scala +++ b/core/src/main/scala/org/apache/spark/SparkThrowableHelper.scala @@ -143,8 +143,9 @@ private[spark] object SparkThrowableHelper { s"Undefined an error message parameter: $messageParameters") } val displayQueryContext = (if (context.isEmpty) "" else "\n") + context + val prefix = if (displayClass.startsWith("_LEGACY_ERROR_TEMP_")) "" else s"[$displayClass] " - s"[$displayClass] $displayMessage$displayQueryContext" + s"$prefix$displayMessage$displayQueryContext" } def getSqlState(errorClass: String): String = { diff --git a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala index 6eb66cd8f18fd..1d1582e438945 100644 --- a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala @@ -124,9 +124,9 @@ class SparkThrowableSuite extends SparkFunSuite { } test("Message format invariants") { - val messageFormats = errorClassToInfoMap.values.toSeq.flatMap { i => - Seq(i.messageFormat) - } + val messageFormats = errorClassToInfoMap + .filterKeys(!_.startsWith("_LEGACY_ERROR_TEMP_")) + .values.toSeq.flatMap { i => Seq(i.messageFormat) } checkCondition(messageFormats, s => s != null) checkIfUnique(messageFormats) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index d2ea8df415108..248837cc3ae14 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2360,7 +2360,10 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit Literal(converter(rawStrippedQualifier)) } catch { case e: NumberFormatException => - throw new ParseException(e.getMessage, ctx) + throw new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0060", + messageParameters = Map("msg" -> e.getMessage), + ctx) } } @@ -2418,7 +2421,10 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit Literal(BigDecimal(raw).underlying()) } catch { case e: AnalysisException => - throw new ParseException(e.message, ctx) + throw new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0061", + messageParameters = Map("msg" -> e.getMessage), + ctx) } } @@ -2616,7 +2622,10 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit IntervalUtils.stringToInterval(UTF8String.concat(kvs: _*)) } catch { case i: IllegalArgumentException => - val e = new ParseException(i.getMessage, ctx) + val e = new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0062", + messageParameters = Map("msg" -> i.getMessage), + ctx) e.setStackTrace(i.getStackTrace) throw e } @@ -2655,7 +2664,10 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } catch { // Handle Exceptions thrown by CalendarInterval case e: IllegalArgumentException => - val pe = new ParseException(e.getMessage, ctx) + val pe = new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0063", + messageParameters = Map("msg" -> e.getMessage), + ctx) pe.setStackTrace(e.getStackTrace) throw pe } 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 4b2ee09804c52..c85a0c2cd45fc 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 @@ -250,13 +250,6 @@ class ParseException( errorSubClass, messageParameters) { - def this(message: String, ctx: ParserRuleContext) = { - this(Option(ParserUtils.command(ctx)), - message, - ParserUtils.position(ctx.getStart), - ParserUtils.position(ctx.getStop)) - } - def this(errorClass: String, messageParameters: Map[String, String], ctx: ParserRuleContext) = this(Option(ParserUtils.command(ctx)), SparkThrowableHelper.getMessage(errorClass, null, messageParameters), @@ -266,6 +259,8 @@ class ParseException( None, messageParameters) + def this(errorClass: String, ctx: ParserRuleContext) = this(errorClass, Map.empty, ctx) + def this( errorClass: String, errorSubClass: String, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala index 048012ac50e9d..6be8d922bb869 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala @@ -135,7 +135,10 @@ object ParserUtils { /** Validate the condition. If it doesn't throw a parse exception. */ def validate(f: => Boolean, message: String, ctx: ParserRuleContext): Unit = { if (!f) { - throw new ParseException(message, ctx) + throw new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0064", + messageParameters = Map("msg" -> message), + ctx) } } 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 51b68383cd0e4..3aa970ffd5468 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 @@ -31,49 +31,52 @@ import org.apache.spark.sql.types.StringType private[sql] object QueryParsingErrors extends QueryErrorsBase { def invalidInsertIntoError(ctx: InsertIntoContext): Throwable = { - new ParseException("Invalid InsertIntoContext", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0001", ctx) } def insertOverwriteDirectoryUnsupportedError(ctx: InsertIntoContext): Throwable = { - new ParseException("INSERT OVERWRITE DIRECTORY is not supported", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0002", ctx) } def columnAliasInOperationNotAllowedError(op: String, ctx: TableAliasContext): Throwable = { - new ParseException(s"Columns aliases are not allowed in $op.", ctx.identifierList()) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0003", + messageParameters = Map("op" -> op), + ctx.identifierList()) } def emptySourceForMergeError(ctx: MergeIntoTableContext): Throwable = { - new ParseException("Empty source for merge: you should specify a source" + - " table/subquery in merge.", ctx.source) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0004", ctx.source) } def unrecognizedMatchedActionError(ctx: MatchedClauseContext): Throwable = { - new ParseException(s"Unrecognized matched action: ${ctx.matchedAction().getText}", + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0005", + messageParameters = Map("matchedAction" -> ctx.matchedAction().getText), ctx.matchedAction()) } def insertedValueNumberNotMatchFieldNumberError(ctx: NotMatchedClauseContext): Throwable = { - new ParseException("The number of inserted values cannot match the fields.", - ctx.notMatchedAction()) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0006", ctx.notMatchedAction()) } def unrecognizedNotMatchedActionError(ctx: NotMatchedClauseContext): Throwable = { - new ParseException(s"Unrecognized not matched action: ${ctx.notMatchedAction().getText}", + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0007", + messageParameters = Map("matchedAction" -> ctx.notMatchedAction().getText), ctx.notMatchedAction()) } def mergeStatementWithoutWhenClauseError(ctx: MergeIntoTableContext): Throwable = { - new ParseException("There must be at least one WHEN clause in a MERGE statement", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0008", ctx) } def nonLastMatchedClauseOmitConditionError(ctx: MergeIntoTableContext): Throwable = { - new ParseException("When there are more than one MATCHED clauses in a MERGE " + - "statement, only the last MATCHED clause can omit the condition.", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0009", ctx) } def nonLastNotMatchedClauseOmitConditionError(ctx: MergeIntoTableContext): Throwable = { - new ParseException("When there are more than one NOT MATCHED clauses in a MERGE " + - "statement, only the last NOT MATCHED clause can omit the condition.", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0010", ctx) } def emptyPartitionKeyError(key: String, ctx: PartitionSpecContext): Throwable = { @@ -85,12 +88,11 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { } def combinationQueryResultClausesUnsupportedError(ctx: QueryOrganizationContext): Throwable = { - new ParseException( - "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0011", ctx) } def distributeByUnsupportedError(ctx: QueryOrganizationContext): Throwable = { - new ParseException("DISTRIBUTE BY is not supported", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0012", ctx) } def transformNotSupportQuantifierError(ctx: ParserRuleContext): Throwable = { @@ -110,7 +112,7 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { } def lateralWithPivotInFromClauseNotAllowedError(ctx: FromClauseContext): Throwable = { - new ParseException("LATERAL cannot be used together with PIVOT in FROM clause", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0013", ctx) } def lateralJoinWithNaturalJoinUnsupportedError(ctx: ParserRuleContext): Throwable = { @@ -178,25 +180,32 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { } def emptyInputForTableSampleError(ctx: ParserRuleContext): Throwable = { - new ParseException("TABLESAMPLE does not accept empty inputs.", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0014", ctx) } def tableSampleByBytesUnsupportedError(msg: String, ctx: SampleMethodContext): Throwable = { - new ParseException(s"TABLESAMPLE($msg) is not supported", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0015", + messageParameters = Map("msg" -> msg), + ctx) } def invalidByteLengthLiteralError(bytesStr: String, ctx: SampleByBytesContext): Throwable = { - new ParseException(s"$bytesStr is not a valid byte length literal, " + - "expected syntax: DIGIT+ ('B' | 'K' | 'M' | 'G')", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0016", + messageParameters = Map("bytesStr" -> bytesStr), + ctx) } def invalidEscapeStringError(ctx: PredicateContext): Throwable = { - new ParseException("Invalid escape string. Escape string must contain only one character.", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0017", ctx) } def trimOptionUnsupportedError(trimOption: Int, ctx: TrimContext): Throwable = { - new ParseException("Function trim doesn't support with " + - s"type $trimOption. Please use BOTH, LEADING or TRAILING as trim type", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0018", + messageParameters = Map("trimOption" -> trimOption.toString), + ctx) } def functionNameUnsupportedError(functionName: String, ctx: ParserRuleContext): Throwable = { @@ -209,58 +218,87 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { def cannotParseValueTypeError( valueType: String, value: String, ctx: TypeConstructorContext): Throwable = { - new ParseException(s"Cannot parse the $valueType value: $value", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0019", + messageParameters = Map("valueType" -> valueType, "value" -> value), + ctx) } def cannotParseIntervalValueError(value: String, ctx: TypeConstructorContext): Throwable = { - new ParseException(s"Cannot parse the INTERVAL value: $value", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0020", + messageParameters = Map("value" -> value), + ctx) } def literalValueTypeUnsupportedError( valueType: String, ctx: TypeConstructorContext): Throwable = { - new ParseException(s"Literals of type '$valueType' are currently not supported.", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0021", + messageParameters = Map("valueType" -> valueType), + ctx) } def parsingValueTypeError( e: IllegalArgumentException, valueType: String, ctx: TypeConstructorContext): Throwable = { val message = Option(e.getMessage).getOrElse(s"Exception parsing $valueType") - new ParseException(message, ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0022", + messageParameters = Map("msg" -> message), + ctx) } def invalidNumericLiteralRangeError(rawStrippedQualifier: String, minValue: BigDecimal, maxValue: BigDecimal, typeName: String, ctx: NumberContext): Throwable = { - new ParseException(s"Numeric literal $rawStrippedQualifier does not " + - s"fit in range [$minValue, $maxValue] for type $typeName", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0023", + messageParameters = Map( + "rawStrippedQualifier" -> rawStrippedQualifier, + "minValue" -> minValue.toString(), + "maxValue" -> maxValue.toString(), + "typeName" -> typeName), + ctx) } def moreThanOneFromToUnitInIntervalLiteralError(ctx: ParserRuleContext): Throwable = { - new ParseException("Can only have a single from-to unit in the interval literal syntax", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0024", ctx) } def invalidIntervalLiteralError(ctx: IntervalContext): Throwable = { - new ParseException("at least one time unit should be given for interval literal", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0025", ctx) } def invalidIntervalFormError(value: String, ctx: MultiUnitsIntervalContext): Throwable = { - new ParseException("Can only use numbers in the interval value part for" + - s" multiple unit value pairs interval form, but got invalid value: $value", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0026", + messageParameters = Map("value" -> value), + ctx) } def invalidFromToUnitValueError(ctx: IntervalValueContext): Throwable = { - new ParseException("The value of from-to unit must be a string", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0027", ctx) } def fromToIntervalUnsupportedError( from: String, to: String, ctx: ParserRuleContext): Throwable = { - new ParseException(s"Intervals FROM $from TO $to are not supported.", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0028", + messageParameters = Map("from" -> from, "to" -> to), + ctx) } def mixedIntervalUnitsError(literal: String, ctx: ParserRuleContext): Throwable = { - new ParseException(s"Cannot mix year-month and day-time fields: $literal", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0029", + messageParameters = Map("literal" -> literal), + ctx) } def dataTypeUnsupportedError(dataType: String, ctx: PrimitiveDataTypeContext): Throwable = { - new ParseException(s"DataType $dataType is not supported.", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0030", + messageParameters = Map("dataType" -> dataType), + ctx) } def charTypeMissingLengthError(dataType: String, ctx: PrimitiveDataTypeContext): Throwable = { @@ -289,7 +327,10 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { } def invalidBucketsNumberError(describe: String, ctx: ApplyTransformContext): Throwable = { - new ParseException(s"Invalid number of buckets: $describe", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0031", + messageParameters = Map("describe" -> describe), + ctx) } def cannotCleanReservedNamespacePropertyError( @@ -321,28 +362,35 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { def duplicatedTablePathsFoundError( pathOne: String, pathTwo: String, ctx: ParserRuleContext): Throwable = { - new ParseException(s"Duplicated table paths found: '$pathOne' and '$pathTwo'. LOCATION" + - s" and the case insensitive key 'path' in OPTIONS are all used to indicate the custom" + - s" table path, you can only specify one of them.", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0032", + messageParameters = Map( + "pathOne" -> pathOne, + "pathTwo" -> pathTwo), + ctx) } def storedAsAndStoredByBothSpecifiedError(ctx: CreateFileFormatContext): Throwable = { - new ParseException("Expected either STORED AS or STORED BY, not both", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0033", ctx) } def operationInHiveStyleCommandUnsupportedError(operation: String, command: String, ctx: StatementContext, msgOpt: Option[String] = None): Throwable = { - val basicError = s"$operation is not supported in Hive-style $command" - val msg = if (msgOpt.isDefined) { - s"$basicError, ${msgOpt.get}." - } else { - basicError - } - new ParseException(msg, ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0034", + messageParameters = Map( + "operation" -> operation, + "command" -> command, + "msg" -> msgOpt.map(m => s", $m").getOrElse("") + ), + ctx) } def operationNotAllowedError(message: String, ctx: ParserRuleContext): Throwable = { - new ParseException(s"Operation not allowed: $message", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0035", + messageParameters = Map("message" -> message), + ctx) } def descColumnForPartitionUnsupportedError(ctx: DescribeRelationContext): Throwable = { @@ -363,13 +411,18 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { } def computeStatisticsNotExpectedError(ctx: IdentifierContext): Throwable = { - new ParseException(s"Expected `NOSCAN` instead of `${ctx.getText}`", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0036", + messageParameters = Map("ctx" -> ctx.getText), + ctx) } def addCatalogInCacheTableAsSelectNotAllowedError( quoted: String, ctx: CacheTableContext): Throwable = { - new ParseException(s"It is not allowed to add catalog/namespace prefix $quoted to " + - "the table name in CACHE TABLE AS SELECT", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0037", + messageParameters = Map("quoted" -> quoted), + ctx) } def showFunctionsUnsupportedError(identifier: String, ctx: IdentifierContext): Throwable = { @@ -392,20 +445,29 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { } def duplicateCteDefinitionNamesError(duplicateNames: String, ctx: CtesContext): Throwable = { - new ParseException(s"CTE definition can't have duplicate names: $duplicateNames.", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0038", + messageParameters = Map("duplicateNames" -> duplicateNames), + ctx) } def sqlStatementUnsupportedError(sqlText: String, position: Origin): Throwable = { - new ParseException(Option(sqlText), "Unsupported SQL statement", position, position) + new ParseException(Option(sqlText), "Unsupported SQL statement", position, position, + Some("_LEGACY_ERROR_TEMP_0039")) } def unquotedIdentifierError(ident: String, ctx: ErrorIdentContext): Throwable = { - new ParseException(s"Possibly unquoted identifier $ident detected. " + - s"Please consider quoting it with back-quotes as `$ident`", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0040", + messageParameters = Map("ident" -> ident), + ctx) } def duplicateClausesError(clauseName: String, ctx: ParserRuleContext): Throwable = { - new ParseException(s"Found duplicate clauses: $clauseName", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0041", + messageParameters = Map("clauseName" -> clauseName), + ctx) } def duplicateKeysError(key: String, ctx: ParserRuleContext): Throwable = { @@ -417,10 +479,7 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { } def unexpectedFormatForSetConfigurationError(ctx: ParserRuleContext): Throwable = { - new ParseException( - "Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include " + - "special characters in key, or include semicolon in value, please use quotes, " + - "e.g., SET `key`=`value`.", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0042", ctx) } def invalidPropertyKeyForSetQuotedConfigurationError( @@ -444,59 +503,59 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { } def unexpectedFormatForResetConfigurationError(ctx: ResetConfigurationContext): Throwable = { - new ParseException( - "Expected format is 'RESET' or 'RESET key'. If you want to include special characters " + - "in key, please use quotes, e.g., RESET `key`.", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0043", ctx) } def intervalValueOutOfRangeError(ctx: IntervalContext): Throwable = { - new ParseException("The interval value must be in the range of [-18, +18] hours" + - " with second precision", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0044", ctx) } def invalidTimeZoneDisplacementValueError(ctx: SetTimeZoneContext): Throwable = { - new ParseException("Invalid time zone displacement value", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0045", ctx) } def createTempTableNotSpecifyProviderError(ctx: CreateTableContext): Throwable = { - new ParseException("CREATE TEMPORARY TABLE without a provider is not allowed.", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0046", ctx) } def rowFormatNotUsedWithStoredAsError(ctx: CreateTableLikeContext): Throwable = { - new ParseException("'ROW FORMAT' must be used with 'STORED AS'", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0047", ctx) } def useDefinedRecordReaderOrWriterClassesError(ctx: ParserRuleContext): Throwable = { - new ParseException( - "Unsupported operation: Used defined record reader/writer classes.", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0048", ctx) } def directoryPathAndOptionsPathBothSpecifiedError(ctx: InsertOverwriteDirContext): Throwable = { - new ParseException( - "Directory path and 'path' in OPTIONS should be specified one, but not both", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0049", ctx) } def unsupportedLocalFileSchemeError(ctx: InsertOverwriteDirContext): Throwable = { - new ParseException("LOCAL is supported only with file: scheme", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0050", ctx) } def invalidGroupingSetError(element: String, ctx: GroupingAnalyticsContext): Throwable = { - new ParseException(s"Empty set in $element grouping sets is not supported.", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0051", + messageParameters = Map("element" -> element), + ctx) } def createViewWithBothIfNotExistsAndReplaceError(ctx: CreateViewContext): Throwable = { - new ParseException("CREATE VIEW with both IF NOT EXISTS and REPLACE is not allowed.", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0052", ctx) } def defineTempViewWithIfNotExistsError(ctx: CreateViewContext): Throwable = { - new ParseException("It is not allowed to define a TEMPORARY view with IF NOT EXISTS.", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0053", ctx) } def notAllowedToAddDBPrefixForTempViewError( database: String, ctx: CreateViewContext): Throwable = { new ParseException( - s"It is not allowed to add database prefix `$database` for the TEMPORARY view name.", ctx) + errorClass = "_LEGACY_ERROR_TEMP_0054", + messageParameters = Map("database" -> database), + ctx) } def createFuncWithBothIfNotExistsAndReplaceError(ctx: CreateFunctionContext): Throwable = { @@ -551,11 +610,15 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { } def unclosedBracketedCommentError(command: String, position: Origin): Throwable = { - new ParseException(Some(command), "Unclosed bracketed comment", position, position) + new ParseException(Some(command), "Unclosed bracketed comment", position, position, + Some("_LEGACY_ERROR_TEMP_0055")) } def invalidTimeTravelSpec(reason: String, ctx: ParserRuleContext): Throwable = { - new ParseException(s"Invalid time travel spec: $reason.", ctx) + new ParseException( + errorClass = "_LEGACY_ERROR_TEMP_0056", + messageParameters = Map("reason" -> reason), + ctx) } def invalidNameForDropTempFunc(name: Seq[String], ctx: ParserRuleContext): Throwable = { @@ -569,15 +632,14 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { } def defaultColumnNotImplementedYetError(ctx: ParserRuleContext): Throwable = { - new ParseException("Support for DEFAULT column values is not implemented yet", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0057", ctx) } def defaultColumnNotEnabledError(ctx: ParserRuleContext): Throwable = { - new ParseException("Support for DEFAULT column values is not allowed", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0058", ctx) } def defaultColumnReferencesNotAllowedInPartitionSpec(ctx: ParserRuleContext): Throwable = { - new ParseException( - "References to DEFAULT column values are not allowed within the PARTITION clause", ctx) + new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0059", ctx) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 80de6f4332c05..58516415e897c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -753,7 +753,7 @@ class ExpressionParserSuite extends AnalysisTest { } // Empty interval statement - intercept("interval", "at least one time unit should be given for interval literal") + intercept("interval", "At least one time unit should be given for interval literal") // Single Intervals. val forms = Seq("", "s") 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 35d60255aba26..73bd301733b91 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 @@ -868,12 +868,19 @@ SELECT CAST(interval 3 month 1 hour AS string) struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot mix year-month and day-time fields: interval 3 month 1 hour(line 1, pos 12) - -== SQL == -SELECT CAST(interval 3 month 1 hour AS string) -------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval 3 month 1 hour" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 35, + "fragment" : "interval 3 month 1 hour" + } ] +} -- !query 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 6d50446a03152..3c16587188933 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 @@ -21,12 +21,20 @@ select date '2020-01-01中文' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 2020-01-01中文(line 1, pos 7) - -== SQL == -select date '2020-01-01中文' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "2020-01-01中文", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "date '2020-01-01中文'" + } ] +} -- !query @@ -61,12 +69,20 @@ select date'015' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 015(line 1, pos 7) - -== SQL == -select date'015' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "015", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "date'015'" + } ] +} -- !query @@ -75,12 +91,20 @@ select date'2021-4294967297-11' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 2021-4294967297-11(line 1, pos 7) - -== SQL == -select date'2021-4294967297-11' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "2021-4294967297-11", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "date'2021-4294967297-11'" + } ] +} -- !query 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 77b3926ae02cd..a39066acbebf7 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 @@ -13,12 +13,19 @@ select interval 4 month 2 weeks 3 microseconds * 1.5 struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot mix year-month and day-time fields: interval 4 month 2 weeks 3 microseconds(line 1, pos 7) - -== SQL == -select interval 4 month 2 weeks 3 microseconds * 1.5 --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval 4 month 2 weeks 3 microseconds" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "interval 4 month 2 weeks 3 microseconds" + } ] +} -- !query @@ -515,12 +522,19 @@ select -interval '-1 month 1 day -1 second' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot mix year-month and day-time fields: interval '-1 month 1 day -1 second'(line 1, pos 8) - -== SQL == -select -interval '-1 month 1 day -1 second' ---------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '-1 month 1 day -1 second'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 43, + "fragment" : "interval '-1 month 1 day -1 second'" + } ] +} -- !query @@ -545,12 +559,19 @@ select -interval -1 month 1 day -1 second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot mix year-month and day-time fields: interval -1 month 1 day -1 second(line 1, pos 8) - -== SQL == -select -interval -1 month 1 day -1 second ---------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval -1 month 1 day -1 second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 41, + "fragment" : "interval -1 month 1 day -1 second" + } ] +} -- !query @@ -575,12 +596,19 @@ select +interval '-1 month 1 day -1 second' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot mix year-month and day-time fields: interval '-1 month 1 day -1 second'(line 1, pos 8) - -== SQL == -select +interval '-1 month 1 day -1 second' ---------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '-1 month 1 day -1 second'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 43, + "fragment" : "interval '-1 month 1 day -1 second'" + } ] +} -- !query @@ -605,12 +633,19 @@ select +interval -1 month 1 day -1 second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot mix year-month and day-time fields: interval -1 month 1 day -1 second(line 1, pos 8) - -== SQL == -select +interval -1 month 1 day -1 second ---------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval -1 month 1 day -1 second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 41, + "fragment" : "interval -1 month 1 day -1 second" + } ] +} -- !query @@ -1039,12 +1074,19 @@ select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisec struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot mix year-month and day-time fields: interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond(line 1, pos 7) - -== SQL == -select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond" + } ] +} -- !query @@ -1077,12 +1119,19 @@ select interval '30' year '25' month '-100' day '40' hour '80' minute '299.88998 struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot mix year-month and day-time fields: interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second(line 1, pos 7) - -== SQL == -select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 92, + "fragment" : "interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second" + } ] +} -- !query @@ -1211,12 +1260,19 @@ select interval '20 15:40:32.99899999' day to hour struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval day to hour: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -select interval '20 15:40:32.99899999' day to hour -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval day to hour: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 50, + "fragment" : "'20 15:40:32.99899999' day to hour" + } ] +} -- !query @@ -1225,12 +1281,19 @@ select interval '20 15:40:32.99899999' day to minute struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]d h:m`, `INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE` when cast to interval day to minute: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -select interval '20 15:40:32.99899999' day to minute -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h:m`, `INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE` when cast to interval day to minute: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 52, + "fragment" : "'20 15:40:32.99899999' day to minute" + } ] +} -- !query @@ -1239,12 +1302,19 @@ select interval '15:40:32.99899999' hour to minute struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE` when cast to interval hour to minute: 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -select interval '15:40:32.99899999' hour to minute -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE` when cast to interval hour to minute: 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 50, + "fragment" : "'15:40:32.99899999' hour to minute" + } ] +} -- !query @@ -1253,12 +1323,19 @@ select interval '15:40.99899999' hour to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -select interval '15:40.99899999' hour to second -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 47, + "fragment" : "'15:40.99899999' hour to second" + } ] +} -- !query @@ -1267,12 +1344,19 @@ select interval '15:40' hour to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -select interval '15:40' hour to second -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 38, + "fragment" : "'15:40' hour to second" + } ] +} -- !query @@ -1281,12 +1365,19 @@ select interval '20 40:32.99899999' minute to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND` when cast to interval minute to second: 20 40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -select interval '20 40:32.99899999' minute to second -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND` when cast to interval minute to second: 20 40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 52, + "fragment" : "'20 40:32.99899999' minute to second" + } ] +} -- !query @@ -1295,12 +1386,19 @@ select interval 10 nanoseconds struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Error parsing ' 10 nanoseconds' to interval, invalid unit 'nanoseconds'(line 1, pos 16) - -== SQL == -select interval 10 nanoseconds -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0062", + "messageParameters" : { + "msg" : "Error parsing ' 10 nanoseconds' to interval, invalid unit 'nanoseconds'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 30, + "fragment" : "10 nanoseconds" + } ] +} -- !query @@ -1389,12 +1487,16 @@ select interval struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -at least one time unit should be given for interval literal(line 1, pos 7) - -== SQL == -select interval --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0025", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "interval" + } ] +} -- !query @@ -1403,12 +1505,19 @@ select interval 1 fake_unit struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Error parsing ' 1 fake_unit' to interval, invalid unit 'fake_unit'(line 1, pos 16) - -== SQL == -select interval 1 fake_unit -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0062", + "messageParameters" : { + "msg" : "Error parsing ' 1 fake_unit' to interval, invalid unit 'fake_unit'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 27, + "fragment" : "1 fake_unit" + } ] +} -- !query @@ -1417,12 +1526,16 @@ select interval 1 year to month struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -The value of from-to unit must be a string(line 1, pos 16) - -== SQL == -select interval 1 year to month -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0027", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 31, + "fragment" : "1 year to month" + } ] +} -- !query @@ -1431,12 +1544,20 @@ select interval '1' year to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Intervals FROM year TO second are not supported.(line 1, pos 16) - -== SQL == -select interval '1' year to second -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0028", + "messageParameters" : { + "from" : "year", + "to" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 34, + "fragment" : "'1' year to second" + } ] +} -- !query @@ -1445,12 +1566,16 @@ select interval '10-9' year to month '2-1' year to month struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) - -== SQL == -select interval '10-9' year to month '2-1' year to month --------------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "interval '10-9' year to month '2-1' year to month" + } ] +} -- !query @@ -1459,12 +1584,16 @@ select interval '10-9' year to month '12:11:10' hour to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) - -== SQL == -select interval '10-9' year to month '12:11:10' hour to second --------------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "interval '10-9' year to month '12:11:10' hour to second" + } ] +} -- !query @@ -1473,12 +1602,16 @@ select interval '1 15:11' day to minute '12:11:10' hour to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 40) - -== SQL == -select interval '1 15:11' day to minute '12:11:10' hour to second -----------------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 65, + "fragment" : "interval '1 15:11' day to minute '12:11:10' hour to second" + } ] +} -- !query @@ -1487,12 +1620,16 @@ select interval 1 year '2-1' year to month struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) - -== SQL == -select interval 1 year '2-1' year to month ------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "interval 1 year '2-1' year to month" + } ] +} -- !query @@ -1501,12 +1638,16 @@ select interval 1 year '12:11:10' hour to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) - -== SQL == -select interval 1 year '12:11:10' hour to second ------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "interval 1 year '12:11:10' hour to second" + } ] +} -- !query @@ -1515,12 +1656,16 @@ select interval '10-9' year to month '1' year struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) - -== SQL == -select interval '10-9' year to month '1' year --------------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "interval '10-9' year to month '1' year" + } ] +} -- !query @@ -1529,12 +1674,16 @@ select interval '12:11:10' hour to second '1' year struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 42) - -== SQL == -select interval '12:11:10' hour to second '1' year -------------------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "interval '12:11:10' hour to second '1' year" + } ] +} -- !query @@ -1619,12 +1768,19 @@ SELECT INTERVAL '178956970-8' YEAR TO MONTH struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Error parsing interval year-month string: integer overflow(line 1, pos 16) - -== SQL == -SELECT INTERVAL '178956970-8' YEAR TO MONTH -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Error parsing interval year-month string: integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 43, + "fragment" : "'178956970-8' YEAR TO MONTH" + } ] +} -- !query @@ -2163,12 +2319,19 @@ select interval '-\t2-2\t' year to month struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match year-month format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval year to month: - 2-2 (line 1, pos 16) - -== SQL == -select interval '-\t2-2\t' year to month -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match year-month format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval year to month: -\t2-2\t" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 40, + "fragment" : "'-\\t2-2\\t' year to month" + } ] +} -- !query @@ -2185,13 +2348,19 @@ select interval '\n-\t10\t 12:34:46.789\t' day to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]d h:m:s.n`, `INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND` when cast to interval day to second: -- 10 12:34:46.789 , set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -select interval '\n-\t10\t 12:34:46.789\t' day to second -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h:m:s.n`, `INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND` when cast to interval day to second: \n-\t10\t 12:34:46.789\t, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 56, + "fragment" : "'\\n-\\t10\\t 12:34:46.789\\t' day to second" + } ] +} -- !query @@ -2200,12 +2369,19 @@ select interval '中文 interval 1 day' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: 中文 interval 1 day(line 1, pos 7) - -== SQL == -select interval '中文 interval 1 day' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "中文 interval 1 day" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "interval '中文 interval 1 day'" + } ] +} -- !query @@ -2214,12 +2390,19 @@ select interval 'interval中文 1 day' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: interval中文 1 day(line 1, pos 7) - -== SQL == -select interval 'interval中文 1 day' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "interval中文 1 day" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "interval 'interval中文 1 day'" + } ] +} -- !query @@ -2228,12 +2411,19 @@ select interval 'interval 1中文day' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: interval 1中文day(line 1, pos 7) - -== SQL == -select interval 'interval 1中文day' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "interval 1中文day" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "interval 'interval 1中文day'" + } ] +} -- !query @@ -2360,12 +2550,19 @@ select interval '+' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: +(line 1, pos 7) - -== SQL == -select interval '+' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "+" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "interval '+'" + } ] +} -- !query @@ -2374,12 +2571,19 @@ select interval '+.' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: +.(line 1, pos 7) - -== SQL == -select interval '+.' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "+." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "interval '+.'" + } ] +} -- !query @@ -2388,12 +2592,19 @@ select interval '1' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: 1(line 1, pos 7) - -== SQL == -select interval '1' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "interval '1'" + } ] +} -- !query @@ -2402,12 +2613,19 @@ select interval '1.2' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: 1.2(line 1, pos 7) - -== SQL == -select interval '1.2' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "1.2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval '1.2'" + } ] +} -- !query @@ -2416,12 +2634,19 @@ select interval '- 2' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: - 2(line 1, pos 7) - -== SQL == -select interval '- 2' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "- 2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval '- 2'" + } ] +} -- !query @@ -2430,12 +2655,19 @@ select interval '1 day -' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: 1 day -(line 1, pos 7) - -== SQL == -select interval '1 day -' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "1 day -" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "interval '1 day -'" + } ] +} -- !query @@ -2444,12 +2676,19 @@ select interval '1 day 1' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: 1 day 1(line 1, pos 7) - -== SQL == -select interval '1 day 1' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "1 day 1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "interval '1 day 1'" + } ] +} -- !query @@ -2458,12 +2697,19 @@ select interval '1 day 2' day struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Can only use numbers in the interval value part for multiple unit value pairs interval form, but got invalid value: 1 day 2(line 1, pos 16) - -== SQL == -select interval '1 day 2' day -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0026", + "messageParameters" : { + "value" : "1 day 2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 29, + "fragment" : "'1 day 2' day" + } ] +} -- !query @@ -2472,12 +2718,19 @@ select interval 'interval 1' day struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Can only use numbers in the interval value part for multiple unit value pairs interval form, but got invalid value: interval 1(line 1, pos 16) - -== SQL == -select interval 'interval 1' day -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0026", + "messageParameters" : { + "value" : "interval 1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 32, + "fragment" : "'interval 1' day" + } ] +} -- !query @@ -2750,12 +3003,19 @@ SELECT INTERVAL '106751992 04' DAY TO HOUR struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -requirement failed: day 106751992 outside range [0, 106751991](line 1, pos 16) - -== SQL == -SELECT INTERVAL '106751992 04' DAY TO HOUR -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: day 106751992 outside range [0, 106751991]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 42, + "fragment" : "'106751992 04' DAY TO HOUR" + } ] +} -- !query @@ -2764,12 +3024,19 @@ SELECT INTERVAL '-106751992 04' DAY TO HOUR struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -requirement failed: day 106751992 outside range [0, 106751991](line 1, pos 16) - -== SQL == -SELECT INTERVAL '-106751992 04' DAY TO HOUR -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: day 106751992 outside range [0, 106751991]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 43, + "fragment" : "'-106751992 04' DAY TO HOUR" + } ] +} -- !query @@ -2778,12 +3045,19 @@ SELECT INTERVAL '2562047789:00' HOUR TO MINUTE struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -requirement failed: hour 2562047789 outside range [0, 2562047788](line 1, pos 16) - -== SQL == -SELECT INTERVAL '2562047789:00' HOUR TO MINUTE -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: hour 2562047789 outside range [0, 2562047788]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 46, + "fragment" : "'2562047789:00' HOUR TO MINUTE" + } ] +} -- !query @@ -2792,12 +3066,19 @@ SELECT INTERVAL '-2562047789:00' HOUR TO MINUTE struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -requirement failed: hour 2562047789 outside range [0, 2562047788](line 1, pos 16) - -== SQL == -SELECT INTERVAL '-2562047789:00' HOUR TO MINUTE -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: hour 2562047789 outside range [0, 2562047788]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 47, + "fragment" : "'-2562047789:00' HOUR TO MINUTE" + } ] +} -- !query @@ -2806,12 +3087,19 @@ SELECT INTERVAL '153722867281:54.775808' MINUTE TO SECOND struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -requirement failed: minute 153722867281 outside range [0, 153722867280](line 1, pos 16) - -== SQL == -SELECT INTERVAL '153722867281:54.775808' MINUTE TO SECOND -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: minute 153722867281 outside range [0, 153722867280]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 57, + "fragment" : "'153722867281:54.775808' MINUTE TO SECOND" + } ] +} -- !query @@ -2820,12 +3108,19 @@ SELECT INTERVAL '-153722867281:54.775808' MINUTE TO SECOND struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -requirement failed: minute 153722867281 outside range [0, 153722867280](line 1, pos 16) - -== SQL == -SELECT INTERVAL '-153722867281:54.775808' MINUTE TO SECOND -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: minute 153722867281 outside range [0, 153722867280]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 58, + "fragment" : "'-153722867281:54.775808' MINUTE TO SECOND" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out index 09827a22b66c8..72b7601f6f5db 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -37,12 +37,22 @@ select 128Y struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Numeric literal 128 does not fit in range [-128, 127] for type tinyint(line 1, pos 7) - -== SQL == -select 128Y --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "127", + "minValue" : "-128", + "rawStrippedQualifier" : "128", + "typeName" : "tinyint" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 11, + "fragment" : "128Y" + } ] +} -- !query @@ -67,12 +77,22 @@ select 32768S struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Numeric literal 32768 does not fit in range [-32768, 32767] for type smallint(line 1, pos 7) - -== SQL == -select 32768S --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "32767", + "minValue" : "-32768", + "rawStrippedQualifier" : "32768", + "typeName" : "smallint" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 13, + "fragment" : "32768S" + } ] +} -- !query @@ -97,12 +117,22 @@ select 9223372036854775808L struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Numeric literal 9223372036854775808 does not fit in range [-9223372036854775808, 9223372036854775807] for type bigint(line 1, pos 7) - -== SQL == -select 9223372036854775808L --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "9223372036854775807", + "minValue" : "-9223372036854775808", + "rawStrippedQualifier" : "9223372036854775808", + "typeName" : "bigint" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "9223372036854775808L" + } ] +} -- !query @@ -183,12 +213,22 @@ select -3.4028235E39f struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Numeric literal -3.4028235E39 does not fit in range [-3.4028234663852886E+38, 3.4028234663852886E+38] for type float(line 1, pos 7) - -== SQL == -select -3.4028235E39f --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "3.4028234663852886E+38", + "minValue" : "-3.4028234663852886E+38", + "rawStrippedQualifier" : "-3.4028235E39", + "typeName" : "float" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "-3.4028235E39f" + } ] +} -- !query @@ -229,12 +269,22 @@ select 1E309, -1E309 struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Numeric literal 1E309 does not fit in range [-1.7976931348623157E+308, 1.7976931348623157E+308] for type double(line 1, pos 7) - -== SQL == -select 1E309, -1E309 --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "1.7976931348623157E+308", + "minValue" : "-1.7976931348623157E+308", + "rawStrippedQualifier" : "1E309", + "typeName" : "double" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "1E309" + } ] +} -- !query @@ -333,12 +383,20 @@ select date 'mar 11 2016' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: mar 11 2016(line 1, pos 7) - -== SQL == -select date 'mar 11 2016' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "mar 11 2016", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date 'mar 11 2016'" + } ] +} -- !query @@ -355,12 +413,20 @@ select timestamp '2016-33-11 20:54:00.000' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the TIMESTAMP value: 2016-33-11 20:54:00.000(line 1, pos 7) - -== SQL == -select timestamp '2016-33-11 20:54:00.000' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "2016-33-11 20:54:00.000", + "valueType" : "TIMESTAMP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "timestamp '2016-33-11 20:54:00.000'" + } ] +} -- !query @@ -369,12 +435,19 @@ select GEO '(10,-6)' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Literals of type 'GEO' are currently not supported.(line 1, pos 7) - -== SQL == -select GEO '(10,-6)' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0021", + "messageParameters" : { + "valueType" : "GEO" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "GEO '(10,-6)'" + } ] +} -- !query @@ -391,12 +464,19 @@ select 1.20E-38BD struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -decimal can only support precision up to 38(line 1, pos 7) - -== SQL == -select 1.20E-38BD --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0061", + "messageParameters" : { + "msg" : "decimal can only support precision up to 38" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "1.20E-38BD" + } ] +} -- !query @@ -413,12 +493,19 @@ select X'XuZ' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -contains illegal character for hexBinary: 0XuZ(line 1, pos 7) - -== SQL == -select X'XuZ' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0022", + "messageParameters" : { + "msg" : "contains illegal character for hexBinary: 0XuZ" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 13, + "fragment" : "X'XuZ'" + } ] +} -- !query 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 bd675df539e77..c80256d0a8b57 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 @@ -13,12 +13,20 @@ select timestamp '2019-01-01中文' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the TIMESTAMP value: 2019-01-01中文(line 1, pos 7) - -== SQL == -select timestamp '2019-01-01中文' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "2019-01-01中文", + "valueType" : "TIMESTAMP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "timestamp '2019-01-01中文'" + } ] +} -- !query @@ -27,12 +35,20 @@ select timestamp'4294967297' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the TIMESTAMP value: 4294967297(line 1, pos 7) - -== SQL == -select timestamp'4294967297' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "4294967297", + "valueType" : "TIMESTAMP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "timestamp'4294967297'" + } ] +} -- !query @@ -41,12 +57,20 @@ select timestamp'2021-01-01T12:30:4294967297.123456' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the TIMESTAMP value: 2021-01-01T12:30:4294967297.123456(line 1, pos 7) - -== SQL == -select timestamp'2021-01-01T12:30:4294967297.123456' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "2021-01-01T12:30:4294967297.123456", + "valueType" : "TIMESTAMP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'" + } ] +} -- !query 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 0b9912160e448..df0405a49d747 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 @@ -350,12 +350,19 @@ SELECT CAST(interval 3 month 1 hour AS string) struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot mix year-month and day-time fields: interval 3 month 1 hour(line 1, pos 12) - -== SQL == -SELECT CAST(interval 3 month 1 hour AS string) -------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval 3 month 1 hour" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 35, + "fragment" : "interval 3 month 1 hour" + } ] +} -- !query 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 68e827ad55cc0..4f35bceb95b7d 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 @@ -23,12 +23,19 @@ ALTER TABLE test_change CHANGE a struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Operation not allowed: ALTER TABLE table CHANGE COLUMN requires a TYPE, a SET/DROP, a COMMENT, or a FIRST/AFTER(line 1, pos 0) - -== SQL == -ALTER TABLE test_change CHANGE a -^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0035", + "messageParameters" : { + "message" : "ALTER TABLE table CHANGE COLUMN requires a TYPE, a SET/DROP, a COMMENT, or a FIRST/AFTER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 32, + "fragment" : "ALTER TABLE test_change CHANGE a" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/comments.sql.out b/sql/core/src/test/resources/sql-tests/results/comments.sql.out index f05d188740fae..0f94aacd813ac 100644 --- a/sql/core/src/test/resources/sql-tests/results/comments.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/comments.sql.out @@ -132,20 +132,9 @@ select 1 as a struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Unclosed bracketed comment(line 3, pos 0) - -== SQL == -/*abc*/ -select 1 as a -/* -^^^ - -2 as b -/*abc*/ -, 3 as c - -/**/ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0055" +} -- !query @@ -163,18 +152,6 @@ select 4 as d struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Unclosed bracketed comment(line 3, pos 0) - -== SQL == -/*abc*/ -select 1 as a -/* -^^^ - -2 as b -/*abc*/ -, 3 as c - -/**/ -select 4 as d +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0055" +} diff --git a/sql/core/src/test/resources/sql-tests/results/cte.sql.out b/sql/core/src/test/resources/sql-tests/results/cte.sql.out index f88fd884ab782..503ddd214d951 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte.sql.out @@ -148,15 +148,19 @@ SELECT * FROM t struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -CTE definition can't have duplicate names: 't'.(line 1, pos 0) - -== SQL == -WITH -^^^ - t(x) AS (SELECT 1), - t(x) AS (SELECT 2) -SELECT * FROM t +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0038", + "messageParameters" : { + "duplicateNames" : "'t'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 63, + "fragment" : "WITH\n t(x) AS (SELECT 1),\n t(x) AS (SELECT 2)\nSELECT * FROM t" + } ] +} -- !query 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 7d19b19e07310..684aa9d0aca50 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 @@ -21,12 +21,20 @@ select date '2020-01-01中文' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 2020-01-01中文(line 1, pos 7) - -== SQL == -select date '2020-01-01中文' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "2020-01-01中文", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "date '2020-01-01中文'" + } ] +} -- !query @@ -59,12 +67,20 @@ select date'015' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 015(line 1, pos 7) - -== SQL == -select date'015' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "015", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "date'015'" + } ] +} -- !query @@ -73,12 +89,20 @@ select date'2021-4294967297-11' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 2021-4294967297-11(line 1, pos 7) - -== SQL == -select date'2021-4294967297-11' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "2021-4294967297-11", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "date'2021-4294967297-11'" + } ] +} -- !query 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 78ea00e5b8dc8..f27d2b20c2d60 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 @@ -21,12 +21,20 @@ select date '2020-01-01中文' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 2020-01-01中文(line 1, pos 7) - -== SQL == -select date '2020-01-01中文' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "2020-01-01中文", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "date '2020-01-01中文'" + } ] +} -- !query @@ -59,12 +67,20 @@ select date'015' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 015(line 1, pos 7) - -== SQL == -select date'015' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "015", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "date'015'" + } ] +} -- !query @@ -73,12 +89,20 @@ select date'2021-4294967297-11' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 2021-4294967297-11(line 1, pos 7) - -== SQL == -select date'2021-4294967297-11' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "2021-4294967297-11", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "date'2021-4294967297-11'" + } ] +} -- !query @@ -1062,12 +1086,20 @@ select timestamp '2019-01-01中文' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the TIMESTAMP value: 2019-01-01中文(line 1, pos 7) - -== SQL == -select timestamp '2019-01-01中文' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "2019-01-01中文", + "valueType" : "TIMESTAMP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "timestamp '2019-01-01中文'" + } ] +} -- !query @@ -1076,12 +1108,20 @@ select timestamp'4294967297' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the TIMESTAMP value: 4294967297(line 1, pos 7) - -== SQL == -select timestamp'4294967297' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "4294967297", + "valueType" : "TIMESTAMP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "timestamp'4294967297'" + } ] +} -- !query @@ -1090,12 +1130,20 @@ select timestamp'2021-01-01T12:30:4294967297.123456' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the TIMESTAMP value: 2021-01-01T12:30:4294967297.123456(line 1, pos 7) - -== SQL == -select timestamp'2021-01-01T12:30:4294967297.123456' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "2021-01-01T12:30:4294967297.123456", + "valueType" : "TIMESTAMP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out index 895cc5bea6da1..8ca6b43bc0d8c 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out @@ -131,12 +131,19 @@ SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Empty set in ROLLUP grouping sets is not supported.(line 1, pos 61) - -== SQL == -SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year, (course, year), ()) ORDER BY course, year --------------------------------------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0051", + "messageParameters" : { + "element" : "ROLLUP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 53, + "stopIndex" : 101, + "fragment" : "GROUP BY ROLLUP(course, year, (course, year), ())" + } ] +} -- !query @@ -193,12 +200,19 @@ SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year, struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Empty set in CUBE grouping sets is not supported.(line 1, pos 61) - -== SQL == -SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year, (course, year), ()) ORDER BY course, year --------------------------------------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0051", + "messageParameters" : { + "element" : "CUBE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 53, + "stopIndex" : 99, + "fragment" : "GROUP BY CUBE(course, year, (course, year), ())" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/ilike-all.sql.out b/sql/core/src/test/resources/sql-tests/results/ilike-all.sql.out index 54dd03d32ea50..96dfe0f97c3c4 100644 --- a/sql/core/src/test/resources/sql-tests/results/ilike-all.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ilike-all.sql.out @@ -129,9 +129,16 @@ SELECT company FROM ilike_any_table WHERE company ILIKE ALL () struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Expected something between '(' and ')'.(line 1, pos 50) - -== SQL == -SELECT company FROM ilike_any_table WHERE company ILIKE ALL () ---------------------------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Expected something between '(' and ')'." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 62, + "fragment" : "ILIKE ALL ()" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/ilike-any.sql.out b/sql/core/src/test/resources/sql-tests/results/ilike-any.sql.out index 91a2f40386457..c8bd38d32462b 100644 --- a/sql/core/src/test/resources/sql-tests/results/ilike-any.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ilike-any.sql.out @@ -135,9 +135,16 @@ SELECT company FROM ilike_any_table WHERE company ILIKE ANY () struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Expected something between '(' and ')'.(line 1, pos 50) - -== SQL == -SELECT company FROM ilike_any_table WHERE company ILIKE ANY () ---------------------------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Expected something between '(' and ')'." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 62, + "fragment" : "ILIKE ANY ()" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 6ee9338151dc3..9b91f9a5b8ca0 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 @@ -13,12 +13,19 @@ select interval 4 month 2 weeks 3 microseconds * 1.5 struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot mix year-month and day-time fields: interval 4 month 2 weeks 3 microseconds(line 1, pos 7) - -== SQL == -select interval 4 month 2 weeks 3 microseconds * 1.5 --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval 4 month 2 weeks 3 microseconds" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "interval 4 month 2 weeks 3 microseconds" + } ] +} -- !query @@ -413,12 +420,19 @@ select -interval '-1 month 1 day -1 second' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot mix year-month and day-time fields: interval '-1 month 1 day -1 second'(line 1, pos 8) - -== SQL == -select -interval '-1 month 1 day -1 second' ---------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '-1 month 1 day -1 second'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 43, + "fragment" : "interval '-1 month 1 day -1 second'" + } ] +} -- !query @@ -443,12 +457,19 @@ select -interval -1 month 1 day -1 second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot mix year-month and day-time fields: interval -1 month 1 day -1 second(line 1, pos 8) - -== SQL == -select -interval -1 month 1 day -1 second ---------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval -1 month 1 day -1 second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 41, + "fragment" : "interval -1 month 1 day -1 second" + } ] +} -- !query @@ -473,12 +494,19 @@ select +interval '-1 month 1 day -1 second' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot mix year-month and day-time fields: interval '-1 month 1 day -1 second'(line 1, pos 8) - -== SQL == -select +interval '-1 month 1 day -1 second' ---------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '-1 month 1 day -1 second'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 43, + "fragment" : "interval '-1 month 1 day -1 second'" + } ] +} -- !query @@ -503,12 +531,19 @@ select +interval -1 month 1 day -1 second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot mix year-month and day-time fields: interval -1 month 1 day -1 second(line 1, pos 8) - -== SQL == -select +interval -1 month 1 day -1 second ---------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval -1 month 1 day -1 second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 41, + "fragment" : "interval -1 month 1 day -1 second" + } ] +} -- !query @@ -920,12 +955,19 @@ select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisec struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot mix year-month and day-time fields: interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond(line 1, pos 7) - -== SQL == -select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond" + } ] +} -- !query @@ -958,12 +1000,19 @@ select interval '30' year '25' month '-100' day '40' hour '80' minute '299.88998 struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot mix year-month and day-time fields: interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second(line 1, pos 7) - -== SQL == -select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 92, + "fragment" : "interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second" + } ] +} -- !query @@ -1092,12 +1141,19 @@ select interval '20 15:40:32.99899999' day to hour struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval day to hour: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -select interval '20 15:40:32.99899999' day to hour -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval day to hour: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 50, + "fragment" : "'20 15:40:32.99899999' day to hour" + } ] +} -- !query @@ -1106,12 +1162,19 @@ select interval '20 15:40:32.99899999' day to minute struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]d h:m`, `INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE` when cast to interval day to minute: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -select interval '20 15:40:32.99899999' day to minute -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h:m`, `INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE` when cast to interval day to minute: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 52, + "fragment" : "'20 15:40:32.99899999' day to minute" + } ] +} -- !query @@ -1120,12 +1183,19 @@ select interval '15:40:32.99899999' hour to minute struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE` when cast to interval hour to minute: 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -select interval '15:40:32.99899999' hour to minute -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE` when cast to interval hour to minute: 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 50, + "fragment" : "'15:40:32.99899999' hour to minute" + } ] +} -- !query @@ -1134,12 +1204,19 @@ select interval '15:40.99899999' hour to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -select interval '15:40.99899999' hour to second -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 47, + "fragment" : "'15:40.99899999' hour to second" + } ] +} -- !query @@ -1148,12 +1225,19 @@ select interval '15:40' hour to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -select interval '15:40' hour to second -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 38, + "fragment" : "'15:40' hour to second" + } ] +} -- !query @@ -1162,12 +1246,19 @@ select interval '20 40:32.99899999' minute to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND` when cast to interval minute to second: 20 40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -select interval '20 40:32.99899999' minute to second -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND` when cast to interval minute to second: 20 40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 52, + "fragment" : "'20 40:32.99899999' minute to second" + } ] +} -- !query @@ -1176,12 +1267,19 @@ select interval 10 nanoseconds struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Error parsing ' 10 nanoseconds' to interval, invalid unit 'nanoseconds'(line 1, pos 16) - -== SQL == -select interval 10 nanoseconds -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0062", + "messageParameters" : { + "msg" : "Error parsing ' 10 nanoseconds' to interval, invalid unit 'nanoseconds'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 30, + "fragment" : "10 nanoseconds" + } ] +} -- !query @@ -1270,12 +1368,16 @@ select interval struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -at least one time unit should be given for interval literal(line 1, pos 7) - -== SQL == -select interval --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0025", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "interval" + } ] +} -- !query @@ -1284,12 +1386,19 @@ select interval 1 fake_unit struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Error parsing ' 1 fake_unit' to interval, invalid unit 'fake_unit'(line 1, pos 16) - -== SQL == -select interval 1 fake_unit -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0062", + "messageParameters" : { + "msg" : "Error parsing ' 1 fake_unit' to interval, invalid unit 'fake_unit'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 27, + "fragment" : "1 fake_unit" + } ] +} -- !query @@ -1298,12 +1407,16 @@ select interval 1 year to month struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -The value of from-to unit must be a string(line 1, pos 16) - -== SQL == -select interval 1 year to month -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0027", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 31, + "fragment" : "1 year to month" + } ] +} -- !query @@ -1312,12 +1425,20 @@ select interval '1' year to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Intervals FROM year TO second are not supported.(line 1, pos 16) - -== SQL == -select interval '1' year to second -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0028", + "messageParameters" : { + "from" : "year", + "to" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 34, + "fragment" : "'1' year to second" + } ] +} -- !query @@ -1326,12 +1447,16 @@ select interval '10-9' year to month '2-1' year to month struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) - -== SQL == -select interval '10-9' year to month '2-1' year to month --------------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "interval '10-9' year to month '2-1' year to month" + } ] +} -- !query @@ -1340,12 +1465,16 @@ select interval '10-9' year to month '12:11:10' hour to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) - -== SQL == -select interval '10-9' year to month '12:11:10' hour to second --------------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "interval '10-9' year to month '12:11:10' hour to second" + } ] +} -- !query @@ -1354,12 +1483,16 @@ select interval '1 15:11' day to minute '12:11:10' hour to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 40) - -== SQL == -select interval '1 15:11' day to minute '12:11:10' hour to second -----------------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 65, + "fragment" : "interval '1 15:11' day to minute '12:11:10' hour to second" + } ] +} -- !query @@ -1368,12 +1501,16 @@ select interval 1 year '2-1' year to month struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) - -== SQL == -select interval 1 year '2-1' year to month ------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "interval 1 year '2-1' year to month" + } ] +} -- !query @@ -1382,12 +1519,16 @@ select interval 1 year '12:11:10' hour to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) - -== SQL == -select interval 1 year '12:11:10' hour to second ------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "interval 1 year '12:11:10' hour to second" + } ] +} -- !query @@ -1396,12 +1537,16 @@ select interval '10-9' year to month '1' year struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) - -== SQL == -select interval '10-9' year to month '1' year --------------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "interval '10-9' year to month '1' year" + } ] +} -- !query @@ -1410,12 +1555,16 @@ select interval '12:11:10' hour to second '1' year struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 42) - -== SQL == -select interval '12:11:10' hour to second '1' year -------------------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "interval '12:11:10' hour to second '1' year" + } ] +} -- !query @@ -1500,12 +1649,19 @@ SELECT INTERVAL '178956970-8' YEAR TO MONTH struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Error parsing interval year-month string: integer overflow(line 1, pos 16) - -== SQL == -SELECT INTERVAL '178956970-8' YEAR TO MONTH -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Error parsing interval year-month string: integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 43, + "fragment" : "'178956970-8' YEAR TO MONTH" + } ] +} -- !query @@ -1976,12 +2132,19 @@ select interval '-\t2-2\t' year to month struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match year-month format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval year to month: - 2-2 (line 1, pos 16) - -== SQL == -select interval '-\t2-2\t' year to month -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match year-month format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval year to month: -\t2-2\t" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 40, + "fragment" : "'-\\t2-2\\t' year to month" + } ] +} -- !query @@ -1998,13 +2161,19 @@ select interval '\n-\t10\t 12:34:46.789\t' day to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]d h:m:s.n`, `INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND` when cast to interval day to second: -- 10 12:34:46.789 , set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -select interval '\n-\t10\t 12:34:46.789\t' day to second -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h:m:s.n`, `INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND` when cast to interval day to second: \n-\t10\t 12:34:46.789\t, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 56, + "fragment" : "'\\n-\\t10\\t 12:34:46.789\\t' day to second" + } ] +} -- !query @@ -2013,12 +2182,19 @@ select interval '中文 interval 1 day' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: 中文 interval 1 day(line 1, pos 7) - -== SQL == -select interval '中文 interval 1 day' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "中文 interval 1 day" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "interval '中文 interval 1 day'" + } ] +} -- !query @@ -2027,12 +2203,19 @@ select interval 'interval中文 1 day' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: interval中文 1 day(line 1, pos 7) - -== SQL == -select interval 'interval中文 1 day' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "interval中文 1 day" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "interval 'interval中文 1 day'" + } ] +} -- !query @@ -2041,12 +2224,19 @@ select interval 'interval 1中文day' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: interval 1中文day(line 1, pos 7) - -== SQL == -select interval 'interval 1中文day' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "interval 1中文day" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "interval 'interval 1中文day'" + } ] +} -- !query @@ -2173,12 +2363,19 @@ select interval '+' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: +(line 1, pos 7) - -== SQL == -select interval '+' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "+" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "interval '+'" + } ] +} -- !query @@ -2187,12 +2384,19 @@ select interval '+.' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: +.(line 1, pos 7) - -== SQL == -select interval '+.' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "+." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "interval '+.'" + } ] +} -- !query @@ -2201,12 +2405,19 @@ select interval '1' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: 1(line 1, pos 7) - -== SQL == -select interval '1' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "interval '1'" + } ] +} -- !query @@ -2215,12 +2426,19 @@ select interval '1.2' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: 1.2(line 1, pos 7) - -== SQL == -select interval '1.2' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "1.2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval '1.2'" + } ] +} -- !query @@ -2229,12 +2447,19 @@ select interval '- 2' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: - 2(line 1, pos 7) - -== SQL == -select interval '- 2' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "- 2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval '- 2'" + } ] +} -- !query @@ -2243,12 +2468,19 @@ select interval '1 day -' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: 1 day -(line 1, pos 7) - -== SQL == -select interval '1 day -' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "1 day -" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "interval '1 day -'" + } ] +} -- !query @@ -2257,12 +2489,19 @@ select interval '1 day 1' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the INTERVAL value: 1 day 1(line 1, pos 7) - -== SQL == -select interval '1 day 1' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0020", + "messageParameters" : { + "value" : "1 day 1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "interval '1 day 1'" + } ] +} -- !query @@ -2271,12 +2510,19 @@ select interval '1 day 2' day struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Can only use numbers in the interval value part for multiple unit value pairs interval form, but got invalid value: 1 day 2(line 1, pos 16) - -== SQL == -select interval '1 day 2' day -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0026", + "messageParameters" : { + "value" : "1 day 2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 29, + "fragment" : "'1 day 2' day" + } ] +} -- !query @@ -2285,12 +2531,19 @@ select interval 'interval 1' day struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Can only use numbers in the interval value part for multiple unit value pairs interval form, but got invalid value: interval 1(line 1, pos 16) - -== SQL == -select interval 'interval 1' day -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0026", + "messageParameters" : { + "value" : "interval 1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 32, + "fragment" : "'interval 1' day" + } ] +} -- !query @@ -2563,12 +2816,19 @@ SELECT INTERVAL '106751992 04' DAY TO HOUR struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -requirement failed: day 106751992 outside range [0, 106751991](line 1, pos 16) - -== SQL == -SELECT INTERVAL '106751992 04' DAY TO HOUR -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: day 106751992 outside range [0, 106751991]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 42, + "fragment" : "'106751992 04' DAY TO HOUR" + } ] +} -- !query @@ -2577,12 +2837,19 @@ SELECT INTERVAL '-106751992 04' DAY TO HOUR struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -requirement failed: day 106751992 outside range [0, 106751991](line 1, pos 16) - -== SQL == -SELECT INTERVAL '-106751992 04' DAY TO HOUR -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: day 106751992 outside range [0, 106751991]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 43, + "fragment" : "'-106751992 04' DAY TO HOUR" + } ] +} -- !query @@ -2591,12 +2858,19 @@ SELECT INTERVAL '2562047789:00' HOUR TO MINUTE struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -requirement failed: hour 2562047789 outside range [0, 2562047788](line 1, pos 16) - -== SQL == -SELECT INTERVAL '2562047789:00' HOUR TO MINUTE -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: hour 2562047789 outside range [0, 2562047788]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 46, + "fragment" : "'2562047789:00' HOUR TO MINUTE" + } ] +} -- !query @@ -2605,12 +2879,19 @@ SELECT INTERVAL '-2562047789:00' HOUR TO MINUTE struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -requirement failed: hour 2562047789 outside range [0, 2562047788](line 1, pos 16) - -== SQL == -SELECT INTERVAL '-2562047789:00' HOUR TO MINUTE -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: hour 2562047789 outside range [0, 2562047788]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 47, + "fragment" : "'-2562047789:00' HOUR TO MINUTE" + } ] +} -- !query @@ -2619,12 +2900,19 @@ SELECT INTERVAL '153722867281:54.775808' MINUTE TO SECOND struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -requirement failed: minute 153722867281 outside range [0, 153722867280](line 1, pos 16) - -== SQL == -SELECT INTERVAL '153722867281:54.775808' MINUTE TO SECOND -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: minute 153722867281 outside range [0, 153722867280]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 57, + "fragment" : "'153722867281:54.775808' MINUTE TO SECOND" + } ] +} -- !query @@ -2633,12 +2921,19 @@ SELECT INTERVAL '-153722867281:54.775808' MINUTE TO SECOND struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -requirement failed: minute 153722867281 outside range [0, 153722867280](line 1, pos 16) - -== SQL == -SELECT INTERVAL '-153722867281:54.775808' MINUTE TO SECOND -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: minute 153722867281 outside range [0, 153722867280]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 58, + "fragment" : "'-153722867281:54.775808' MINUTE TO SECOND" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/like-all.sql.out b/sql/core/src/test/resources/sql-tests/results/like-all.sql.out index 3a252d84ba155..ce3cc3a0c0f00 100644 --- a/sql/core/src/test/resources/sql-tests/results/like-all.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/like-all.sql.out @@ -129,9 +129,16 @@ SELECT company FROM like_all_table WHERE company LIKE ALL () struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Expected something between '(' and ')'.(line 1, pos 49) - -== SQL == -SELECT company FROM like_all_table WHERE company LIKE ALL () --------------------------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Expected something between '(' and ')'." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 50, + "stopIndex" : 60, + "fragment" : "LIKE ALL ()" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/like-any.sql.out b/sql/core/src/test/resources/sql-tests/results/like-any.sql.out index ecf7b6e7bbcc3..8fcfbb607a356 100644 --- a/sql/core/src/test/resources/sql-tests/results/like-any.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/like-any.sql.out @@ -135,9 +135,16 @@ SELECT company FROM like_any_table WHERE company LIKE ANY () struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Expected something between '(' and ')'.(line 1, pos 49) - -== SQL == -SELECT company FROM like_any_table WHERE company LIKE ANY () --------------------------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Expected something between '(' and ')'." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 50, + "stopIndex" : 60, + "fragment" : "LIKE ANY ()" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index 09827a22b66c8..72b7601f6f5db 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -37,12 +37,22 @@ select 128Y struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Numeric literal 128 does not fit in range [-128, 127] for type tinyint(line 1, pos 7) - -== SQL == -select 128Y --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "127", + "minValue" : "-128", + "rawStrippedQualifier" : "128", + "typeName" : "tinyint" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 11, + "fragment" : "128Y" + } ] +} -- !query @@ -67,12 +77,22 @@ select 32768S struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Numeric literal 32768 does not fit in range [-32768, 32767] for type smallint(line 1, pos 7) - -== SQL == -select 32768S --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "32767", + "minValue" : "-32768", + "rawStrippedQualifier" : "32768", + "typeName" : "smallint" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 13, + "fragment" : "32768S" + } ] +} -- !query @@ -97,12 +117,22 @@ select 9223372036854775808L struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Numeric literal 9223372036854775808 does not fit in range [-9223372036854775808, 9223372036854775807] for type bigint(line 1, pos 7) - -== SQL == -select 9223372036854775808L --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "9223372036854775807", + "minValue" : "-9223372036854775808", + "rawStrippedQualifier" : "9223372036854775808", + "typeName" : "bigint" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "9223372036854775808L" + } ] +} -- !query @@ -183,12 +213,22 @@ select -3.4028235E39f struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Numeric literal -3.4028235E39 does not fit in range [-3.4028234663852886E+38, 3.4028234663852886E+38] for type float(line 1, pos 7) - -== SQL == -select -3.4028235E39f --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "3.4028234663852886E+38", + "minValue" : "-3.4028234663852886E+38", + "rawStrippedQualifier" : "-3.4028235E39", + "typeName" : "float" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "-3.4028235E39f" + } ] +} -- !query @@ -229,12 +269,22 @@ select 1E309, -1E309 struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Numeric literal 1E309 does not fit in range [-1.7976931348623157E+308, 1.7976931348623157E+308] for type double(line 1, pos 7) - -== SQL == -select 1E309, -1E309 --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "1.7976931348623157E+308", + "minValue" : "-1.7976931348623157E+308", + "rawStrippedQualifier" : "1E309", + "typeName" : "double" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "1E309" + } ] +} -- !query @@ -333,12 +383,20 @@ select date 'mar 11 2016' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: mar 11 2016(line 1, pos 7) - -== SQL == -select date 'mar 11 2016' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "mar 11 2016", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date 'mar 11 2016'" + } ] +} -- !query @@ -355,12 +413,20 @@ select timestamp '2016-33-11 20:54:00.000' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the TIMESTAMP value: 2016-33-11 20:54:00.000(line 1, pos 7) - -== SQL == -select timestamp '2016-33-11 20:54:00.000' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "2016-33-11 20:54:00.000", + "valueType" : "TIMESTAMP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "timestamp '2016-33-11 20:54:00.000'" + } ] +} -- !query @@ -369,12 +435,19 @@ select GEO '(10,-6)' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Literals of type 'GEO' are currently not supported.(line 1, pos 7) - -== SQL == -select GEO '(10,-6)' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0021", + "messageParameters" : { + "valueType" : "GEO" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "GEO '(10,-6)'" + } ] +} -- !query @@ -391,12 +464,19 @@ select 1.20E-38BD struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -decimal can only support precision up to 38(line 1, pos 7) - -== SQL == -select 1.20E-38BD --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0061", + "messageParameters" : { + "msg" : "decimal can only support precision up to 38" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "1.20E-38BD" + } ] +} -- !query @@ -413,12 +493,19 @@ select X'XuZ' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -contains illegal character for hexBinary: 0XuZ(line 1, pos 7) - -== SQL == -select X'XuZ' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0022", + "messageParameters" : { + "msg" : "contains illegal character for hexBinary: 0XuZ" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 13, + "fragment" : "X'XuZ'" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out index 7edda980be3fd..73ac127b9e056 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out @@ -198,12 +198,20 @@ SELECT date '1999 Jan 08' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 1999 Jan 08(line 1, pos 7) - -== SQL == -SELECT date '1999 Jan 08' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "1999 Jan 08", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date '1999 Jan 08'" + } ] +} -- !query @@ -212,12 +220,20 @@ SELECT date '1999 08 Jan' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 1999 08 Jan(line 1, pos 7) - -== SQL == -SELECT date '1999 08 Jan' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "1999 08 Jan", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date '1999 08 Jan'" + } ] +} -- !query @@ -242,12 +258,20 @@ SELECT date '1999 01 08' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 1999 01 08(line 1, pos 7) - -== SQL == -SELECT date '1999 01 08' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "1999 01 08", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "date '1999 01 08'" + } ] +} -- !query @@ -256,12 +280,20 @@ SELECT date '1999 08 01' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 1999 08 01(line 1, pos 7) - -== SQL == -SELECT date '1999 08 01' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "1999 08 01", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "date '1999 08 01'" + } ] +} -- !query @@ -278,12 +310,20 @@ SELECT date '1999 Jan 08' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 1999 Jan 08(line 1, pos 7) - -== SQL == -SELECT date '1999 Jan 08' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "1999 Jan 08", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date '1999 Jan 08'" + } ] +} -- !query @@ -292,12 +332,20 @@ SELECT date '1999 08 Jan' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 1999 08 Jan(line 1, pos 7) - -== SQL == -SELECT date '1999 08 Jan' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "1999 08 Jan", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date '1999 08 Jan'" + } ] +} -- !query @@ -322,12 +370,20 @@ SELECT date '1999 01 08' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 1999 01 08(line 1, pos 7) - -== SQL == -SELECT date '1999 01 08' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "1999 01 08", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "date '1999 01 08'" + } ] +} -- !query @@ -336,12 +392,20 @@ SELECT date '1999 08 01' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 1999 08 01(line 1, pos 7) - -== SQL == -SELECT date '1999 08 01' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "1999 08 01", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "date '1999 08 01'" + } ] +} -- !query @@ -366,12 +430,20 @@ SELECT date '1999 Jan 08' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 1999 Jan 08(line 1, pos 7) - -== SQL == -SELECT date '1999 Jan 08' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "1999 Jan 08", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date '1999 Jan 08'" + } ] +} -- !query @@ -380,12 +452,20 @@ SELECT date '1999 08 Jan' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 1999 08 Jan(line 1, pos 7) - -== SQL == -SELECT date '1999 08 Jan' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "1999 08 Jan", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date '1999 08 Jan'" + } ] +} -- !query @@ -410,12 +490,20 @@ SELECT date '1999 01 08' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 1999 01 08(line 1, pos 7) - -== SQL == -SELECT date '1999 01 08' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "1999 01 08", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "date '1999 01 08'" + } ] +} -- !query @@ -424,12 +512,20 @@ SELECT date '1999 08 01' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the DATE value: 1999 08 01(line 1, pos 7) - -== SQL == -SELECT date '1999 08 01' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "1999 08 01", + "valueType" : "DATE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "date '1999 08 01'" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out index 04df07bff577b..bff615e22af0b 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out @@ -101,12 +101,19 @@ SELECT interval '1 2:03' day to hour struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval day to hour: 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -SELECT interval '1 2:03' day to hour -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval day to hour: 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 36, + "fragment" : "'1 2:03' day to hour" + } ] +} -- !query @@ -115,12 +122,19 @@ SELECT interval '1 2:03:04' day to hour struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval day to hour: 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -SELECT interval '1 2:03:04' day to hour -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval day to hour: 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 39, + "fragment" : "'1 2:03:04' day to hour" + } ] +} -- !query @@ -137,12 +151,19 @@ SELECT interval '1 2:03:04' day to minute struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]d h:m`, `INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE` when cast to interval day to minute: 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -SELECT interval '1 2:03:04' day to minute -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h:m`, `INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE` when cast to interval day to minute: 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 41, + "fragment" : "'1 2:03:04' day to minute" + } ] +} -- !query @@ -151,12 +172,19 @@ SELECT interval '1 2:03' day to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]d h:m:s.n`, `INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND` when cast to interval day to second: 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -SELECT interval '1 2:03' day to second -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h:m:s.n`, `INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND` when cast to interval day to second: 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 38, + "fragment" : "'1 2:03' day to second" + } ] +} -- !query @@ -173,12 +201,19 @@ SELECT interval '1 2:03' hour to minute struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE` when cast to interval hour to minute: 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -SELECT interval '1 2:03' hour to minute -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE` when cast to interval hour to minute: 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 39, + "fragment" : "'1 2:03' hour to minute" + } ] +} -- !query @@ -187,12 +222,19 @@ SELECT interval '1 2:03:04' hour to minute struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE` when cast to interval hour to minute: 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -SELECT interval '1 2:03:04' hour to minute -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE` when cast to interval hour to minute: 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 42, + "fragment" : "'1 2:03:04' hour to minute" + } ] +} -- !query @@ -201,12 +243,19 @@ SELECT interval '1 2:03' hour to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -SELECT interval '1 2:03' hour to second -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 39, + "fragment" : "'1 2:03' hour to second" + } ] +} -- !query @@ -215,12 +264,19 @@ SELECT interval '1 2:03:04' hour to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -SELECT interval '1 2:03:04' hour to second -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 42, + "fragment" : "'1 2:03:04' hour to second" + } ] +} -- !query @@ -229,12 +285,19 @@ SELECT interval '1 2:03' minute to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND` when cast to interval minute to second: 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -SELECT interval '1 2:03' minute to second -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND` when cast to interval minute to second: 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 41, + "fragment" : "'1 2:03' minute to second" + } ] +} -- !query @@ -243,9 +306,16 @@ SELECT interval '1 2:03:04' minute to second struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Interval string does not match day-time format of `[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND` when cast to interval minute to second: 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) - -== SQL == -SELECT interval '1 2:03:04' minute to second -----------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND` when cast to interval minute to second: 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 44, + "fragment" : "'1 2:03:04' minute to second" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out index 0a4e2f179d8c2..f6f3fcee20bac 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out @@ -346,12 +346,19 @@ create table foo (with baz) struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -DataType baz is not supported.(line 1, pos 23) - -== SQL == -create table foo (with baz) ------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0030", + "messageParameters" : { + "dataType" : "baz" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 26, + "fragment" : "baz" + } ] +} -- !query @@ -360,12 +367,19 @@ create table foo (with ordinality) struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -DataType ordinality is not supported.(line 1, pos 23) - -== SQL == -create table foo (with ordinality) ------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0030", + "messageParameters" : { + "dataType" : "ordinality" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 33, + "fragment" : "ordinality" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/tablesample-negative.sql.out b/sql/core/src/test/resources/sql-tests/results/tablesample-negative.sql.out index 9624cdfe7b9d9..fd852992397d2 100644 --- a/sql/core/src/test/resources/sql-tests/results/tablesample-negative.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/tablesample-negative.sql.out @@ -29,12 +29,19 @@ SELECT mydb1.t1 FROM t1 TABLESAMPLE (-1 PERCENT) struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Sampling fraction (-0.01) must be on interval [0, 1](line 1, pos 24) - -== SQL == -SELECT mydb1.t1 FROM t1 TABLESAMPLE (-1 PERCENT) -------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Sampling fraction (-0.01) must be on interval [0, 1]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 48, + "fragment" : "TABLESAMPLE (-1 PERCENT)" + } ] +} -- !query @@ -43,12 +50,19 @@ SELECT mydb1.t1 FROM t1 TABLESAMPLE (101 PERCENT) struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Sampling fraction (1.01) must be on interval [0, 1](line 1, pos 24) - -== SQL == -SELECT mydb1.t1 FROM t1 TABLESAMPLE (101 PERCENT) -------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Sampling fraction (1.01) must be on interval [0, 1]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 49, + "fragment" : "TABLESAMPLE (101 PERCENT)" + } ] +} -- !query 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 d0819faafb2dc..6400636be8510 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 @@ -13,12 +13,20 @@ select timestamp '2019-01-01中文' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the TIMESTAMP value: 2019-01-01中文(line 1, pos 7) - -== SQL == -select timestamp '2019-01-01中文' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "2019-01-01中文", + "valueType" : "TIMESTAMP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "timestamp '2019-01-01中文'" + } ] +} -- !query @@ -27,12 +35,20 @@ select timestamp'4294967297' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the TIMESTAMP value: 4294967297(line 1, pos 7) - -== SQL == -select timestamp'4294967297' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "4294967297", + "valueType" : "TIMESTAMP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "timestamp'4294967297'" + } ] +} -- !query @@ -41,12 +57,20 @@ select timestamp'2021-01-01T12:30:4294967297.123456' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the TIMESTAMP value: 2021-01-01T12:30:4294967297.123456(line 1, pos 7) - -== SQL == -select timestamp'2021-01-01T12:30:4294967297.123456' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "2021-01-01T12:30:4294967297.123456", + "valueType" : "TIMESTAMP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'" + } ] +} -- !query 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 269b6d5ff0c23..ea3332f1e5639 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 @@ -13,12 +13,20 @@ select timestamp '2019-01-01中文' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the TIMESTAMP value: 2019-01-01中文(line 1, pos 7) - -== SQL == -select timestamp '2019-01-01中文' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "2019-01-01中文", + "valueType" : "TIMESTAMP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "timestamp '2019-01-01中文'" + } ] +} -- !query @@ -27,12 +35,20 @@ select timestamp'4294967297' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the TIMESTAMP value: 4294967297(line 1, pos 7) - -== SQL == -select timestamp'4294967297' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "4294967297", + "valueType" : "TIMESTAMP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "timestamp'4294967297'" + } ] +} -- !query @@ -41,12 +57,20 @@ select timestamp'2021-01-01T12:30:4294967297.123456' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the TIMESTAMP value: 2021-01-01T12:30:4294967297.123456(line 1, pos 7) - -== SQL == -select timestamp'2021-01-01T12:30:4294967297.123456' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "2021-01-01T12:30:4294967297.123456", + "valueType" : "TIMESTAMP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'" + } ] +} -- !query 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 fde1c3169d666..a8db85911419c 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 @@ -13,12 +13,20 @@ select timestamp '2019-01-01中文' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the TIMESTAMP value: 2019-01-01中文(line 1, pos 7) - -== SQL == -select timestamp '2019-01-01中文' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "2019-01-01中文", + "valueType" : "TIMESTAMP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "timestamp '2019-01-01中文'" + } ] +} -- !query @@ -27,12 +35,20 @@ select timestamp'4294967297' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the TIMESTAMP value: 4294967297(line 1, pos 7) - -== SQL == -select timestamp'4294967297' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "4294967297", + "valueType" : "TIMESTAMP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "timestamp'4294967297'" + } ] +} -- !query @@ -41,12 +57,20 @@ select timestamp'2021-01-01T12:30:4294967297.123456' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the TIMESTAMP value: 2021-01-01T12:30:4294967297.123456(line 1, pos 7) - -== SQL == -select timestamp'2021-01-01T12:30:4294967297.123456' --------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0019", + "messageParameters" : { + "value" : "2021-01-01T12:30:4294967297.123456", + "valueType" : "TIMESTAMP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/timezone.sql.out b/sql/core/src/test/resources/sql-tests/results/timezone.sql.out index 7dbdb174be09a..8c74d8e7f27db 100644 --- a/sql/core/src/test/resources/sql-tests/results/timezone.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timezone.sql.out @@ -45,12 +45,16 @@ SET TIME ZONE struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Invalid time zone displacement value(line 1, pos 0) - -== SQL == -SET TIME ZONE -^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0045", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 13, + "fragment" : "SET TIME ZONE" + } ] +} -- !query @@ -68,12 +72,16 @@ SET TIME ZONE INTERVAL 3 DAYS struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -The interval value must be in the range of [-18, +18] hours with second precision(line 1, pos 14) - -== SQL == -SET TIME ZONE INTERVAL 3 DAYS ---------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "SET TIME ZONE INTERVAL 3 DAYS" + } ] +} -- !query @@ -82,12 +90,16 @@ SET TIME ZONE INTERVAL 24 HOURS struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -The interval value must be in the range of [-18, +18] hours with second precision(line 1, pos 14) - -== SQL == -SET TIME ZONE INTERVAL 24 HOURS ---------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 31, + "fragment" : "SET TIME ZONE INTERVAL 24 HOURS" + } ] +} -- !query @@ -96,12 +108,16 @@ SET TIME ZONE INTERVAL '19:40:32' HOUR TO SECOND struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -The interval value must be in the range of [-18, +18] hours with second precision(line 1, pos 14) - -== SQL == -SET TIME ZONE INTERVAL '19:40:32' HOUR TO SECOND ---------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 48, + "fragment" : "SET TIME ZONE INTERVAL '19:40:32' HOUR TO SECOND" + } ] +} -- !query @@ -110,12 +126,16 @@ SET TIME ZONE INTERVAL 10 HOURS 'GMT+1' struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Invalid time zone displacement value(line 1, pos 0) - -== SQL == -SET TIME ZONE INTERVAL 10 HOURS 'GMT+1' -^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0045", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 39, + "fragment" : "SET TIME ZONE INTERVAL 10 HOURS 'GMT+1'" + } ] +} -- !query @@ -124,9 +144,13 @@ SET TIME ZONE INTERVAL 10 HOURS 1 MILLISECOND struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -The interval value must be in the range of [-18, +18] hours with second precision(line 1, pos 14) - -== SQL == -SET TIME ZONE INTERVAL 10 HOURS 1 MILLISECOND ---------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 45, + "fragment" : "SET TIME ZONE INTERVAL 10 HOURS 1 MILLISECOND" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/transform.sql.out b/sql/core/src/test/resources/sql-tests/results/transform.sql.out index 190e8a4cd68f4..13fd30488548a 100644 --- a/sql/core/src/test/resources/sql-tests/results/transform.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/transform.sql.out @@ -400,36 +400,19 @@ SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -LINES TERMINATED BY only supports newline '\n' right now: @(line 3, pos 4) - -== SQL == -SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( - SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) - ROW FORMAT DELIMITED -----^^^ - FIELDS TERMINATED BY ',' - LINES TERMINATED BY '@' - NULL DEFINED AS 'NULL' - USING 'cat' AS ( - a string, - b string, - c string, - d string, - e string, - f string, - g string, - h string, - i string, - j string, - k string, - l string) - ROW FORMAT DELIMITED - FIELDS TERMINATED BY ',' - LINES TERMINATED BY '@' - NULL DEFINED AS 'NULL' - FROM t -) tmp +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "LINES TERMINATED BY only supports newline '\\n' right now: @" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 69, + "stopIndex" : 560, + "fragment" : "SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l)\n ROW FORMAT DELIMITED\n FIELDS TERMINATED BY ','\n LINES TERMINATED BY '@'\n NULL DEFINED AS 'NULL'\n USING 'cat' AS (\n a string,\n b string,\n c string,\n d string,\n e string,\n f string,\n g string,\n h string,\n i string,\n j string,\n k string,\n l string)\n ROW FORMAT DELIMITED\n FIELDS TERMINATED BY ','\n LINES TERMINATED BY '@'\n NULL DEFINED AS 'NULL'\n FROM t" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-analytics.sql.out index 4b16407e1a0c0..b8c94b19d8136 100644 --- a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-analytics.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-analytics.sql.out @@ -131,12 +131,19 @@ SELECT course, year, udaf(earnings) FROM courseSales GROUP BY ROLLUP(course, yea struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Empty set in ROLLUP grouping sets is not supported.(line 1, pos 62) - -== SQL == -SELECT course, year, udaf(earnings) FROM courseSales GROUP BY ROLLUP(course, year, (course, year), ()) ORDER BY course, year ---------------------------------------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0051", + "messageParameters" : { + "element" : "ROLLUP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 54, + "stopIndex" : 102, + "fragment" : "GROUP BY ROLLUP(course, year, (course, year), ())" + } ] +} -- !query @@ -193,12 +200,19 @@ SELECT course, year, udaf(earnings) FROM courseSales GROUP BY CUBE(course, year, struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Empty set in CUBE grouping sets is not supported.(line 1, pos 62) - -== SQL == -SELECT course, year, udaf(earnings) FROM courseSales GROUP BY CUBE(course, year, (course, year), ()) ORDER BY course, year ---------------------------------------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0051", + "messageParameters" : { + "element" : "CUBE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 54, + "stopIndex" : 100, + "fragment" : "GROUP BY CUBE(course, year, (course, year), ())" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out index 076c7dd6a1562..94e10b3a06774 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out @@ -238,13 +238,19 @@ RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cat struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Frame bound value must be a literal.(line 2, pos 30) - -== SQL == -SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY udf(val) -RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val(val) -------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Frame bound value must be a literal." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 111, + "stopIndex" : 132, + "fragment" : "current_date PRECEDING" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index 59345a4773ffb..2b358d221e69b 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -401,13 +401,19 @@ RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cat struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException - -Frame bound value must be a literal.(line 2, pos 30) - -== SQL == -SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val -RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val -------------------------------^^^ +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Frame bound value must be a literal." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 96, + "stopIndex" : 117, + "fragment" : "current_date PRECEDING" + } ] +} -- !query