diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 3518765efd09..220d2aa8104c 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -2030,7 +2030,7 @@ }, "IDENTIFIER_TOO_MANY_NAME_PARTS" : { "message" : [ - " is not a valid identifier as it has more than 2 name parts." + " is not a valid identifier as it has more than name parts." ], "sqlState" : "42601" }, @@ -8539,11 +8539,6 @@ "Failed to merge incompatible schemas and ." ] }, - "_LEGACY_ERROR_TEMP_2096" : { - "message" : [ - " is not supported temporarily." - ] - }, "_LEGACY_ERROR_TEMP_2097" : { "message" : [ "Could not execute broadcast in secs. You can increase the timeout for broadcasts via or disable broadcast join by setting to -1 or remove the broadcast hint if it exists in your code." diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 8ccac6a39d2c..9d942bc60159 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -46,6 +46,13 @@ options { tokenVocab = SqlBaseLexer; } * When true, parameter markers are allowed everywhere a literal is supported. */ public boolean parameter_substitution_enabled = true; + + /** + * When false (default), IDENTIFIER('literal') is resolved to an identifier at parse time (identifier-lite). + * When true, only the legacy IDENTIFIER(expression) function syntax is allowed. + * Controlled by spark.sql.legacy.identifierClause configuration. + */ + public boolean legacy_identifier_clause_only = false; } compoundOrSingleStatement @@ -92,7 +99,7 @@ sqlStateValue ; declareConditionStatement - : DECLARE multipartIdentifier CONDITION (FOR SQLSTATE VALUE? sqlStateValue)? + : DECLARE strictIdentifier CONDITION (FOR SQLSTATE VALUE? sqlStateValue)? ; conditionValue @@ -125,11 +132,11 @@ repeatStatement ; leaveStatement - : LEAVE multipartIdentifier + : LEAVE strictIdentifier ; iterateStatement - : ITERATE multipartIdentifier + : ITERATE strictIdentifier ; caseStatement @@ -144,7 +151,7 @@ loopStatement ; forStatement - : beginLabel? FOR (multipartIdentifier AS)? query DO compoundBody END FOR endLabel? + : beginLabel? FOR (strictIdentifier AS)? query DO compoundBody END FOR endLabel? ; singleStatement @@ -152,11 +159,11 @@ singleStatement ; beginLabel - : multipartIdentifier COLON + : strictIdentifier COLON ; endLabel - : multipartIdentifier + : strictIdentifier ; singleExpression @@ -321,7 +328,7 @@ statement | SHOW VIEWS ((FROM | IN) identifierReference)? (LIKE? pattern=stringLit)? #showViews | SHOW PARTITIONS identifierReference partitionSpec? #showPartitions - | SHOW identifier? FUNCTIONS ((FROM | IN) ns=identifierReference)? + | SHOW functionScope=simpleIdentifier? FUNCTIONS ((FROM | IN) ns=identifierReference)? (LIKE? (legacy=multipartIdentifier | pattern=stringLit))? #showFunctions | SHOW PROCEDURES ((FROM | IN) identifierReference)? #showProcedures | SHOW CREATE TABLE identifierReference (AS SERDE)? #showCreateTable @@ -833,8 +840,8 @@ hint ; hintStatement - : hintName=identifier - | hintName=identifier LEFT_PAREN parameters+=primaryExpression (COMMA parameters+=primaryExpression)* RIGHT_PAREN + : hintName=simpleIdentifier + | hintName=simpleIdentifier LEFT_PAREN parameters+=primaryExpression (COMMA parameters+=primaryExpression)* RIGHT_PAREN ; fromClause @@ -1241,7 +1248,7 @@ primaryExpression | identifier #columnReference | base=primaryExpression DOT fieldName=identifier #dereference | LEFT_PAREN expression RIGHT_PAREN #parenthesizedExpression - | EXTRACT LEFT_PAREN field=identifier FROM source=valueExpression RIGHT_PAREN #extract + | EXTRACT LEFT_PAREN field=simpleIdentifier FROM source=valueExpression RIGHT_PAREN #extract | (SUBSTR | SUBSTRING) LEFT_PAREN str=valueExpression (FROM | COMMA) pos=valueExpression ((FOR | COMMA) len=valueExpression)? RIGHT_PAREN #substring | TRIM LEFT_PAREN trimOption=(BOTH | LEADING | TRAILING)? (trimStr=valueExpression)? @@ -1297,7 +1304,7 @@ constant ; namedParameterMarker - : COLON identifier + : COLON simpleIdentifier ; comparisonOperator : EQ | NEQ | NEQJ | LT | LTE | GT | GTE | NSEQ @@ -1599,13 +1606,32 @@ identifier | {!SQL_standard_keyword_behavior}? strictNonReserved ; +// simpleIdentifier: like identifier but without IDENTIFIER('literal') support +// Use this for contexts where IDENTIFIER() syntax is not appropriate: +// - Named parameters (:param_name) +// - Extract field names (EXTRACT(field FROM ...)) +// - Other keyword-like or string-like uses +simpleIdentifier + : simpleStrictIdentifier + | {!SQL_standard_keyword_behavior}? strictNonReserved + ; + strictIdentifier : IDENTIFIER #unquotedIdentifier | quotedIdentifier #quotedIdentifierAlternative + | {!legacy_identifier_clause_only}? IDENTIFIER_KW LEFT_PAREN stringLit RIGHT_PAREN #identifierLiteral | {SQL_standard_keyword_behavior}? ansiNonReserved #unquotedIdentifier | {!SQL_standard_keyword_behavior}? nonReserved #unquotedIdentifier ; +// simpleStrictIdentifier: like strictIdentifier but without IDENTIFIER('literal') support +simpleStrictIdentifier + : IDENTIFIER #simpleUnquotedIdentifier + | quotedIdentifier #simpleQuotedIdentifierAlternative + | {SQL_standard_keyword_behavior}? ansiNonReserved #simpleUnquotedIdentifier + | {!SQL_standard_keyword_behavior}? nonReserved #simpleUnquotedIdentifier + ; + quotedIdentifier : BACKQUOTED_IDENTIFIER | {double_quoted_identifiers}? DOUBLEQUOTED_STRING diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala index 73767990bd3a..212c80a3cb43 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala @@ -20,7 +20,7 @@ import java.util.Locale import scala.jdk.CollectionConverters._ -import org.antlr.v4.runtime.Token +import org.antlr.v4.runtime.{ParserRuleContext, Token} import org.antlr.v4.runtime.tree.ParseTree import org.apache.spark.SparkException @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.util.CollationFactory import org.apache.spark.sql.catalyst.util.SparkParserUtils.{string, withOrigin} import org.apache.spark.sql.connector.catalog.IdentityColumnSpec -import org.apache.spark.sql.errors.QueryParsingErrors +import org.apache.spark.sql.errors.{DataTypeErrorsBase, QueryParsingErrors} import org.apache.spark.sql.internal.SqlApiConf import org.apache.spark.sql.types.{ArrayType, BinaryType, BooleanType, ByteType, CalendarIntervalType, CharType, DataType, DateType, DayTimeIntervalType, DecimalType, DoubleType, FloatType, GeographyType, GeometryType, IntegerType, LongType, MapType, MetadataBuilder, NullType, ShortType, StringType, StructField, StructType, TimestampNTZType, TimestampType, TimeType, VarcharType, VariantType, YearMonthIntervalType} @@ -60,12 +60,52 @@ import org.apache.spark.sql.types.{ArrayType, BinaryType, BooleanType, ByteType, * * @see * [[org.apache.spark.sql.catalyst.parser.AstBuilder]] for the full SQL statement parser + * + * ==CRITICAL: Extracting Identifier Names== + * + * When extracting identifier names from parser contexts, you MUST use the helper methods provided + * by this class instead of calling ctx.getText() directly: + * + * - '''getIdentifierText(ctx)''': For single identifiers (column names, aliases, window names) + * - '''getIdentifierParts(ctx)''': For qualified identifiers (table names, schema.table) + * + * '''DO NOT use ctx.getText() or ctx.identifier.getText()''' directly! These methods do not + * handle the IDENTIFIER('literal') syntax and will cause incorrect behavior. + * + * The IDENTIFIER('literal') syntax allows string literals to be used as identifiers at parse time + * (e.g., IDENTIFIER('my_col') resolves to the identifier my_col). If you use getText(), you'll + * get the raw text "IDENTIFIER('my_col')" instead of "my_col", breaking the feature. + * + * Example: + * {{{ + * // WRONG - does not handle IDENTIFIER('literal'): + * val name = ctx.identifier.getText + * SubqueryAlias(ctx.name.getText, plan) + * + * // CORRECT - handles both regular identifiers and IDENTIFIER('literal'): + * val name = getIdentifierText(ctx.identifier) + * SubqueryAlias(getIdentifierText(ctx.name), plan) + * }}} */ -class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] { +class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with DataTypeErrorsBase { protected def typedVisit[T](ctx: ParseTree): T = { ctx.accept(this).asInstanceOf[T] } + /** + * Public helper to extract identifier parts from a context. This is exposed as public to allow + * utility classes like ParserUtils to reuse the identifier resolution logic without duplicating + * code. + * + * @param ctx + * The parser context containing the identifier. + * @return + * Sequence of identifier parts. + */ + def extractIdentifierParts(ctx: ParserRuleContext): Seq[String] = { + getIdentifierParts(ctx) + } + override def visitSingleDataType(ctx: SingleDataTypeContext): DataType = withOrigin(ctx) { typedVisit[DataType](ctx.dataType) } @@ -161,11 +201,89 @@ class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] { } /** - * Create a multi-part identifier. + * Parse a string into a multi-part identifier. Subclasses should override this method to + * provide proper multi-part identifier parsing with access to a full SQL parser. + * + * For example, in AstBuilder, this would parse "`catalog`.`schema`.`table`" into Seq("catalog", + * "schema", "table"). + * + * This method is only called when parsing IDENTIFIER('literal') where the literal contains a + * qualified identifier (e.g., IDENTIFIER('schema.table')). Since DataTypeAstBuilder only parses + * data types (not full SQL with qualified table names), this should never be called in + * practice. The base implementation throws an error to catch unexpected usage. + * + * @param identifier + * The identifier string to parse, potentially containing dots and backticks. + * @return + * Sequence of identifier parts. + */ + protected def parseMultipartIdentifier(identifier: String): Seq[String] = { + throw SparkException.internalError( + "parseMultipartIdentifier must be overridden by subclasses. " + + s"Attempted to parse: $identifier") + } + + /** + * Get the identifier parts from a context, handling both regular identifiers and + * IDENTIFIER('literal'). This method is used to support identifier-lite syntax where + * IDENTIFIER('string') is folded at parse time. For qualified identifiers like + * IDENTIFIER('`catalog`.`schema`'), this will parse the string and return multiple parts. + * + * Subclasses should override this method to provide actual parsing logic. + */ + protected def getIdentifierParts(ctx: ParserRuleContext): Seq[String] = { + ctx match { + case idCtx: IdentifierContext => + // identifier can be either strictIdentifier or strictNonReserved. + // Recursively process the strictIdentifier. + Option(idCtx.strictIdentifier()).map(getIdentifierParts).getOrElse(Seq(ctx.getText)) + + case idLitCtx: IdentifierLiteralContext => + // For IDENTIFIER('literal') in strictIdentifier. + val literalValue = string(visitStringLit(idLitCtx.stringLit())) + // Parse the string to handle qualified identifiers like "`cat`.`schema`". + parseMultipartIdentifier(literalValue) + + case errCapture: ErrorCapturingIdentifierContext => + // Regular identifier with errorCapturingIdentifierExtra. + // Need to recursively handle identifier which might itself be IDENTIFIER('literal'). + Option(errCapture.identifier()) + .flatMap(id => Option(id.strictIdentifier()).map(getIdentifierParts)) + .getOrElse(Seq(ctx.getText)) + + case _ => + // For regular identifiers, just return the text as a single part. + Seq(ctx.getText) + } + } + + /** + * Get the text of a SINGLE identifier, handling both regular identifiers and + * IDENTIFIER('literal'). This method REQUIRES that the identifier be unqualified (single part + * only). If IDENTIFIER('qualified.name') is used where a single identifier is required, this + * will error. + */ + protected def getIdentifierText(ctx: ParserRuleContext): String = { + val parts = getIdentifierParts(ctx) + if (parts.size > 1) { + throw new ParseException( + errorClass = "IDENTIFIER_TOO_MANY_NAME_PARTS", + messageParameters = Map("identifier" -> toSQLId(parts), "limit" -> "1"), + ctx) + } + parts.head + } + + /** + * Create a multi-part identifier. Handles identifier-lite with qualified identifiers like + * IDENTIFIER('`cat`.`schema`').table */ override def visitMultipartIdentifier(ctx: MultipartIdentifierContext): Seq[String] = withOrigin(ctx) { - ctx.parts.asScala.map(_.getText).toSeq + // Each part is an errorCapturingIdentifier (which wraps identifier). + // getIdentifierParts recursively handles IDENTIFIER('literal') syntax through + // identifier -> strictIdentifier -> identifierLiteral. + ctx.parts.asScala.flatMap(getIdentifierParts).toSeq } /** @@ -351,7 +469,7 @@ class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] { } StructField( - name = colName.getText, + name = getIdentifierText(colName), dataType = typedVisit[DataType](ctx.dataType), nullable = NULL == null, metadata = builder.build()) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/SubstituteParmsAstBuilder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/SubstituteParmsAstBuilder.scala index 8beeb9b17d4c..f32c1d6f3836 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/SubstituteParmsAstBuilder.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/SubstituteParmsAstBuilder.scala @@ -81,7 +81,8 @@ class SubstituteParmsAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] { */ override def visitNamedParameterLiteral(ctx: NamedParameterLiteralContext): AnyRef = withOrigin(ctx) { - val paramName = ctx.namedParameterMarker().identifier().getText + // Named parameters use simpleIdentifier, so .getText() is correct. + val paramName = ctx.namedParameterMarker().simpleIdentifier().getText namedParams += paramName // Calculate the location of the entire parameter (including the colon) @@ -117,7 +118,8 @@ class SubstituteParmsAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] { */ override def visitNamedParameterMarkerRule(ctx: NamedParameterMarkerRuleContext): AnyRef = withOrigin(ctx) { - val paramName = ctx.namedParameterMarker().identifier().getText + // Named parameters use simpleIdentifier, so .getText() is correct. + val paramName = ctx.namedParameterMarker().simpleIdentifier().getText namedParams += paramName // Calculate the location of the entire parameter (including the colon) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala index e2e320be3654..32270df0a988 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala @@ -430,7 +430,15 @@ case class UnclosedCommentProcessor(command: String, tokenStream: CommonTokenStr } object DataTypeParser extends AbstractParser { - override protected def astBuilder: DataTypeAstBuilder = new DataTypeAstBuilder + override protected def astBuilder: DataTypeAstBuilder = new DataTypeAstBuilder { + // DataTypeParser only parses data types, not full SQL. + // Multi-part identifiers should not appear in IDENTIFIER() within type definitions. + override protected def parseMultipartIdentifier(identifier: String): Seq[String] = { + throw SparkException.internalError( + "DataTypeParser does not support multi-part identifiers in IDENTIFIER(). " + + s"Attempted to parse: $identifier") + } + } } object AbstractParser extends Logging { @@ -476,6 +484,7 @@ object AbstractParser extends Logging { parser.SQL_standard_keyword_behavior = conf.enforceReservedKeywords parser.double_quoted_identifiers = conf.doubleQuotedIdentifiers parser.parameter_substitution_enabled = !conf.legacyParameterSubstitutionConstantsOnly + parser.legacy_identifier_clause_only = conf.legacyIdentifierClauseOnly } /** diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala index 630f274a621e..553161ea2db0 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala @@ -477,7 +477,7 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { ctx) } - def showFunctionsUnsupportedError(identifier: String, ctx: IdentifierContext): Throwable = { + def showFunctionsUnsupportedError(identifier: String, ctx: ParserRuleContext): Throwable = { new ParseException( errorClass = "INVALID_SQL_SYNTAX.SHOW_FUNCTIONS_INVALID_SCOPE", messageParameters = Map("scope" -> toSQLId(identifier)), diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala index f715f8f9ed8c..fd2a2c0ee5b2 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala @@ -51,6 +51,7 @@ private[sql] trait SqlApiConf { def parserDfaCacheFlushThreshold: Int def parserDfaCacheFlushRatio: Double def legacyParameterSubstitutionConstantsOnly: Boolean + def legacyIdentifierClauseOnly: Boolean } private[sql] object SqlApiConf { @@ -104,4 +105,5 @@ private[sql] object DefaultSqlApiConf extends SqlApiConf { override def parserDfaCacheFlushThreshold: Int = -1 override def parserDfaCacheFlushRatio: Double = -1.0 override def legacyParameterSubstitutionConstantsOnly: Boolean = false + override def legacyIdentifierClauseOnly: Boolean = false } 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 971633b9a46a..d1d4a6b8c980 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 @@ -118,6 +118,15 @@ class AstBuilder extends DataTypeAstBuilder } } + /** + * Override to provide actual multi-part identifier parsing using CatalystSqlParser. This allows + * the base class to handle IDENTIFIER('qualified.identifier') without needing special case + * logic in getIdentifierParts. + */ + override protected def parseMultipartIdentifier(identifier: String): Seq[String] = { + CatalystSqlParser.parseMultipartIdentifier(identifier) + } + /** * Retrieves the original input text for a given parser context, preserving all whitespace and * formatting. @@ -255,12 +264,6 @@ class AstBuilder extends DataTypeAstBuilder private def visitDeclareConditionStatementImpl( ctx: DeclareConditionStatementContext): ErrorCondition = { - // Qualified user defined condition name is not allowed. - if (ctx.multipartIdentifier().parts.size() > 1) { - throw SqlScriptingErrors - .conditionCannotBeQualified(CurrentOrigin.get, ctx.multipartIdentifier().getText) - } - // If SQLSTATE is not provided, default to 45000. val sqlState = Option(ctx.sqlStateValue()) .map(sqlStateValueContext => string(visitStringLit(sqlStateValueContext.stringLit()))) @@ -269,7 +272,7 @@ class AstBuilder extends DataTypeAstBuilder assertSqlState(sqlState) // Get condition name. - val conditionName = visitMultipartIdentifier(ctx.multipartIdentifier()).head + val conditionName = getIdentifierText(ctx.strictIdentifier()) assertConditionName(conditionName) @@ -561,15 +564,15 @@ class AstBuilder extends DataTypeAstBuilder val query = withOrigin(queryCtx) { SingleStatement(visitQuery(queryCtx)) } - parsingCtx.labelContext.enterForScope(Option(ctx.multipartIdentifier())) - val varName = Option(ctx.multipartIdentifier()).map(_.getText) + parsingCtx.labelContext.enterForScope(Option(ctx.strictIdentifier())) + val varName = Option(ctx.strictIdentifier()).map(getIdentifierText) val body = visitCompoundBodyImpl( ctx.compoundBody(), None, parsingCtx, isScope = false ) - parsingCtx.labelContext.exitForScope(Option(ctx.multipartIdentifier())) + parsingCtx.labelContext.exitForScope(Option(ctx.strictIdentifier())) parsingCtx.labelContext.exitLabeledScope(Option(ctx.beginLabel())) ForStatement(query, varName, body, Some(labelText)) @@ -580,26 +583,26 @@ class AstBuilder extends DataTypeAstBuilder ctx match { case c: BeginEndCompoundBlockContext if Option(c.beginLabel()).exists { b => - b.multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) + getIdentifierText(b.strictIdentifier()).toLowerCase(Locale.ROOT).equals(label) } => if (isIterate) { throw SqlScriptingErrors.invalidIterateLabelUsageForCompound(CurrentOrigin.get, label) } true case c: WhileStatementContext if Option(c.beginLabel()).exists { b => - b.multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) + getIdentifierText(b.strictIdentifier()).toLowerCase(Locale.ROOT).equals(label) } => true case c: RepeatStatementContext if Option(c.beginLabel()).exists { b => - b.multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) + getIdentifierText(b.strictIdentifier()).toLowerCase(Locale.ROOT).equals(label) } => true case c: LoopStatementContext if Option(c.beginLabel()).exists { b => - b.multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) + getIdentifierText(b.strictIdentifier()).toLowerCase(Locale.ROOT).equals(label) } => true case c: ForStatementContext if Option(c.beginLabel()).exists { b => - b.multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) + getIdentifierText(b.strictIdentifier()).toLowerCase(Locale.ROOT).equals(label) } => true case _ => false } @@ -607,7 +610,7 @@ class AstBuilder extends DataTypeAstBuilder override def visitLeaveStatement(ctx: LeaveStatementContext): LeaveStatement = withOrigin(ctx) { - val labelText = ctx.multipartIdentifier().getText.toLowerCase(Locale.ROOT) + val labelText = getIdentifierText(ctx.strictIdentifier()).toLowerCase(Locale.ROOT) var parentCtx = ctx.parent while (Option(parentCtx).isDefined) { @@ -623,7 +626,7 @@ class AstBuilder extends DataTypeAstBuilder override def visitIterateStatement(ctx: IterateStatementContext): IterateStatement = withOrigin(ctx) { - val labelText = ctx.multipartIdentifier().getText.toLowerCase(Locale.ROOT) + val labelText = getIdentifierText(ctx.strictIdentifier()).toLowerCase(Locale.ROOT) var parentCtx = ctx.parent while (Option(parentCtx).isDefined) { @@ -797,7 +800,8 @@ class AstBuilder extends DataTypeAstBuilder (columnAliases, plan) => UnresolvedSubqueryColumnAliases(visitIdentifierList(columnAliases), plan) ) - SubqueryAlias(ctx.name.getText, subQuery) + // Use getIdentifierText to handle both regular identifiers and IDENTIFIER('literal') + SubqueryAlias(getIdentifierText(ctx.name), subQuery) } /** @@ -1786,7 +1790,8 @@ class AstBuilder extends DataTypeAstBuilder // Collect all window specifications defined in the WINDOW clause. val baseWindowTuples = ctx.namedWindow.asScala.map { wCtx => - (wCtx.name.getText, typedVisit[WindowSpec](wCtx.windowSpec)) + // Use getIdentifierText to handle both regular identifiers and IDENTIFIER('literal') + (getIdentifierText(wCtx.name), typedVisit[WindowSpec](wCtx.windowSpec)) } baseWindowTuples.groupBy(_._1).foreach { kv => if (kv._2.size > 1) { @@ -1927,6 +1932,7 @@ class AstBuilder extends DataTypeAstBuilder query: LogicalPlan): LogicalPlan = withOrigin(ctx) { var plan = query ctx.hintStatements.asScala.reverse.foreach { stmt => + // Hint names use simpleIdentifier, so .getText() is correct. plan = UnresolvedHint(stmt.hintName.getText, stmt.parameters.asScala.map(expression).toSeq, plan) } @@ -1974,17 +1980,18 @@ class AstBuilder extends DataTypeAstBuilder // this is needed to create unpivot and to filter unpivot for nulls further down val valueColumnNames = Option(ctx.unpivotOperator().unpivotSingleValueColumnClause()) - .map(_.unpivotValueColumn().identifier().getText) + .map(vc => getIdentifierText(vc.unpivotValueColumn().identifier())) .map(Seq(_)) .getOrElse( Option(ctx.unpivotOperator().unpivotMultiValueColumnClause()) - .map(_.unpivotValueColumns.asScala.map(_.identifier().getText).toSeq) + .map(_.unpivotValueColumns.asScala.map(vc => + getIdentifierText(vc.identifier())).toSeq) .get ) val unpivot = if (ctx.unpivotOperator().unpivotSingleValueColumnClause() != null) { val unpivotClause = ctx.unpivotOperator().unpivotSingleValueColumnClause() - val variableColumnName = unpivotClause.unpivotNameColumn().identifier().getText + val variableColumnName = getIdentifierText(unpivotClause.unpivotNameColumn().identifier()) val (unpivotColumns, unpivotAliases) = unpivotClause.unpivotColumns.asScala.map(visitUnpivotColumnAndAlias).toSeq.unzip @@ -1999,7 +2006,7 @@ class AstBuilder extends DataTypeAstBuilder ) } else { val unpivotClause = ctx.unpivotOperator().unpivotMultiValueColumnClause() - val variableColumnName = unpivotClause.unpivotNameColumn().identifier().getText + val variableColumnName = getIdentifierText(unpivotClause.unpivotNameColumn().identifier()) val (unpivotColumns, unpivotAliases) = unpivotClause.unpivotColumnSets.asScala.map(visitUnpivotColumnSet).toSeq.unzip @@ -2043,7 +2050,7 @@ class AstBuilder extends DataTypeAstBuilder override def visitUnpivotColumnAndAlias(ctx: UnpivotColumnAndAliasContext): (NamedExpression, Option[String]) = withOrigin(ctx) { val attr = visitUnpivotColumn(ctx.unpivotColumn()) - val alias = Option(ctx.unpivotAlias()).map(_.errorCapturingIdentifier().getText) + val alias = Option(ctx.unpivotAlias()).map(a => getIdentifierText(a.errorCapturingIdentifier())) (attr, alias) } @@ -2055,7 +2062,8 @@ class AstBuilder extends DataTypeAstBuilder (Seq[NamedExpression], Option[String]) = withOrigin(ctx) { val exprs = ctx.unpivotColumns.asScala.map(visitUnpivotColumn).toSeq - val alias = Option(ctx.unpivotAlias()).map(_.errorCapturingIdentifier().getText) + val alias = + Option(ctx.unpivotAlias()).map(a => getIdentifierText(a.errorCapturingIdentifier())) (exprs, alias) } @@ -2071,9 +2079,9 @@ class AstBuilder extends DataTypeAstBuilder unrequiredChildIndex = Nil, outer = ctx.OUTER != null, // scalastyle:off caselocale - Some(ctx.tblName.getText.toLowerCase), + Some(getIdentifierText(ctx.tblName).toLowerCase), // scalastyle:on caselocale - ctx.colName.asScala.map(_.getText).map(UnresolvedAttribute.quoted).toSeq, + ctx.colName.asScala.map(getIdentifierText).map(UnresolvedAttribute.quoted).toSeq, query) } @@ -2514,7 +2522,8 @@ class AstBuilder extends DataTypeAstBuilder * Create an alias ([[SubqueryAlias]]) for a [[LogicalPlan]]. */ private def aliasPlan(alias: ParserRuleContext, plan: LogicalPlan): LogicalPlan = { - SubqueryAlias(alias.getText, plan) + // Use getIdentifierText to handle both regular identifiers and IDENTIFIER('literal') + SubqueryAlias(getIdentifierText(alias), plan) } /** @@ -2544,9 +2553,11 @@ class AstBuilder extends DataTypeAstBuilder /** * Create a Sequence of Strings for an identifier list. + * Each identifier must be unqualified. + * Handles both regular identifiers and IDENTIFIER('literal'). */ override def visitIdentifierSeq(ctx: IdentifierSeqContext): Seq[String] = withOrigin(ctx) { - ctx.ident.asScala.map(_.getText).toSeq + ctx.ident.asScala.map(id => getIdentifierText(id)).toSeq } /* ******************************************************************************************** @@ -2554,18 +2565,20 @@ class AstBuilder extends DataTypeAstBuilder * ******************************************************************************************** */ /** * Create a [[TableIdentifier]] from a 'tableName' or 'databaseName'.'tableName' pattern. + * Handles identifier-lite with qualified identifiers. */ override def visitTableIdentifier( ctx: TableIdentifierContext): TableIdentifier = withOrigin(ctx) { - TableIdentifier(ctx.table.getText, Option(ctx.db).map(_.getText)) + TableIdentifier(getIdentifierText(ctx.table), Option(ctx.db).map(getIdentifierText)) } /** * Create a [[FunctionIdentifier]] from a 'functionName' or 'databaseName'.'functionName' pattern. + * Handles identifier-lite with qualified identifiers. */ override def visitFunctionIdentifier( ctx: FunctionIdentifierContext): FunctionIdentifier = withOrigin(ctx) { - FunctionIdentifier(ctx.function.getText, Option(ctx.db).map(_.getText)) + FunctionIdentifier(getIdentifierText(ctx.function), Option(ctx.db).map(getIdentifierText)) } /* ******************************************************************************************** @@ -2639,7 +2652,8 @@ class AstBuilder extends DataTypeAstBuilder override def visitNamedExpression(ctx: NamedExpressionContext): Expression = withOrigin(ctx) { val e = expression(ctx.expression) if (ctx.name != null) { - Alias(e, ctx.name.getText)() + // Use getIdentifierText to handle both regular identifiers and IDENTIFIER('literal') + Alias(e, getIdentifierText(ctx.name))() } else if (ctx.identifierList != null) { MultiAlias(e, visitIdentifierList(ctx.identifierList)) } else { @@ -2972,7 +2986,8 @@ class AstBuilder extends DataTypeAstBuilder } } else { // If the parser is not in ansi mode, we should return `UnresolvedAttribute`, in case there - // are columns named `CURRENT_DATE` or `CURRENT_TIMESTAMP` or `CURRENT_TIME` + // are columns named `CURRENT_DATE` or `CURRENT_TIMESTAMP` or `CURRENT_TIME`. + // ctx.name is a token, not an identifier context. UnresolvedAttribute.quoted(ctx.name.getText) } } @@ -3217,7 +3232,8 @@ class AstBuilder extends DataTypeAstBuilder * Create a reference to a window frame, i.e. [[WindowSpecReference]]. */ override def visitWindowRef(ctx: WindowRefContext): WindowSpecReference = withOrigin(ctx) { - WindowSpecReference(ctx.name.getText) + // Use getIdentifierText to handle both regular identifiers and IDENTIFIER('literal') + WindowSpecReference(getIdentifierText(ctx.name)) } /** @@ -3353,9 +3369,11 @@ class AstBuilder extends DataTypeAstBuilder * it can be [[UnresolvedExtractValue]]. */ override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) { - val attr = ctx.fieldName.getText + // Use getIdentifierText to handle both regular identifiers and IDENTIFIER('literal') + val attr = getIdentifierText(ctx.fieldName) expression(ctx.base) match { case unresolved_attr @ UnresolvedAttribute(nameParts) => + // For regex check, we need the original text before identifier-lite resolution ctx.fieldName.getStart.getText match { case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName && @@ -3393,13 +3411,17 @@ class AstBuilder extends DataTypeAstBuilder * quoted in `` */ override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) { + // For regex check, we need the original text before identifier-lite resolution ctx.getStart.getText match { case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName && isRegex(columnNameRegex) && canApplyRegex(ctx) => UnresolvedRegex(columnNameRegex, None, conf.caseSensitiveAnalysis) case _ => - UnresolvedAttribute.quoted(ctx.getText) + // Use getIdentifierParts to handle IDENTIFIER('literal') correctly + // This allows IDENTIFIER('t').c1 to work like t.c1 + val parts = getIdentifierParts(ctx.identifier()) + UnresolvedAttribute(parts) } } @@ -4046,7 +4068,7 @@ class AstBuilder extends DataTypeAstBuilder ctx: ColDefinitionContext): ColumnAndConstraint = withOrigin(ctx) { import ctx._ - val name: String = colName.getText + val name: String = getIdentifierText(colName) // Check that no duplicates exist among any CREATE TABLE column options specified. var nullable = true var defaultExpression: Option[DefaultExpressionContext] = None @@ -4118,7 +4140,7 @@ class AstBuilder extends DataTypeAstBuilder ctx: ColumnConstraintDefinitionContext): TableConstraint = { withOrigin(ctx) { val name = if (ctx.name != null) { - ctx.name.getText + getIdentifierText(ctx.name) } else { null } @@ -5460,7 +5482,8 @@ class AstBuilder extends DataTypeAstBuilder invalidStatement("ALTER TABLE ... PARTITION ... CHANGE COLUMN", ctx) } val columnNameParts = typedVisit[Seq[String]](ctx.colName) - if (!conf.resolver(columnNameParts.last, ctx.colType().colName.getText)) { + if (!conf.resolver(columnNameParts.last, + getIdentifierText(ctx.colType().colName))) { throw QueryParsingErrors.operationInHiveStyleCommandUnsupportedError("Renaming column", "ALTER COLUMN", ctx, Some("please run RENAME COLUMN instead")) } @@ -5587,7 +5610,7 @@ class AstBuilder extends DataTypeAstBuilder ctx: TableConstraintDefinitionContext): TableConstraint = withOrigin(ctx) { val name = if (ctx.name != null) { - ctx.name.getText + getIdentifierText(ctx.name) } else { null } @@ -5691,7 +5714,7 @@ class AstBuilder extends DataTypeAstBuilder ctx.identifierReference, "ALTER TABLE ... DROP CONSTRAINT") DropConstraint( table, - ctx.name.getText, + getIdentifierText(ctx.name), ifExists = ctx.EXISTS() != null, cascade = ctx.CASCADE() != null) } @@ -5924,8 +5947,6 @@ class AstBuilder extends DataTypeAstBuilder * }}} */ override def visitCacheTable(ctx: CacheTableContext): LogicalPlan = withOrigin(ctx) { - import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ - val query = Option(ctx.query).map(plan) withIdentClause(ctx.identifierReference, query.toSeq, (ident, children) => { if (query.isDefined && ident.length > 1) { @@ -6303,12 +6324,14 @@ class AstBuilder extends DataTypeAstBuilder * Create a plan for a SHOW FUNCTIONS command. */ override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) { - val (userScope, systemScope) = Option(ctx.identifier) - .map(_.getText.toLowerCase(Locale.ROOT)) match { + // Function scope uses simpleIdentifier, so .getText() is correct. + val scope = Option(ctx.functionScope) + val (userScope, systemScope) = scope.map(_.getText.toLowerCase(Locale.ROOT)) match { case None | Some("all") => (true, true) case Some("system") => (false, true) case Some("user") => (true, false) - case Some(x) => throw QueryParsingErrors.showFunctionsUnsupportedError(x, ctx.identifier()) + case Some(x) => + throw QueryParsingErrors.showFunctionsUnsupportedError(x, ctx.functionScope) } val legacy = Option(ctx.legacy).map(visitMultipartIdentifier) @@ -6435,11 +6458,13 @@ class AstBuilder extends DataTypeAstBuilder */ override def visitTimestampadd(ctx: TimestampaddContext): Expression = withOrigin(ctx) { if (ctx.invalidUnit != null) { + // ctx.name and ctx.invalidUnit are tokens, not identifier contexts. throw QueryParsingErrors.invalidDatetimeUnitError( ctx, ctx.name.getText, ctx.invalidUnit.getText) } else { + // ctx.unit is a token, not an identifier context. TimestampAdd(ctx.unit.getText, expression(ctx.unitsAmount), expression(ctx.timestamp)) } } @@ -6449,11 +6474,13 @@ class AstBuilder extends DataTypeAstBuilder */ override def visitTimestampdiff(ctx: TimestampdiffContext): Expression = withOrigin(ctx) { if (ctx.invalidUnit != null) { + // ctx.name and ctx.invalidUnit are tokens, not identifier contexts. throw QueryParsingErrors.invalidDatetimeUnitError( ctx, ctx.name.getText, ctx.invalidUnit.getText) } else { + // ctx.unit is a token, not an identifier context. TimestampDiff(ctx.unit.getText, expression(ctx.startTimestamp), expression(ctx.endTimestamp)) } } @@ -6463,7 +6490,8 @@ class AstBuilder extends DataTypeAstBuilder * */ override def visitNamedParameterLiteral( ctx: NamedParameterLiteralContext): Expression = withOrigin(ctx) { - NamedParameter(ctx.namedParameterMarker().identifier().getText) + // Named parameters use simpleIdentifier, so .getText() is correct. + NamedParameter(ctx.namedParameterMarker().simpleIdentifier().getText) } /** 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 a19b4cca2817..336db1382f89 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 @@ -28,7 +28,7 @@ import org.antlr.v4.runtime.tree.{ParseTree, TerminalNodeImpl} import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.analysis.UnresolvedIdentifier -import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BeginLabelContext, EndLabelContext, MultipartIdentifierContext} +import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BeginLabelContext, EndLabelContext, StrictIdentifierContext} import org.apache.spark.sql.catalyst.plans.logical.{CreateVariable, ErrorCondition} import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.catalyst.util.SparkParserUtils @@ -279,34 +279,49 @@ class SqlScriptingLabelContext { * @param beginLabelCtx Begin label context. * @param endLabelCtx The end label context. */ + /** + * Get label text from label context, handling IDENTIFIER() syntax. + */ + private def getLabelText(ctx: ParserRuleContext): String = { + val astBuilder = new DataTypeAstBuilder { + override protected def parseMultipartIdentifier(identifier: String): Seq[String] = { + CatalystSqlParser.parseMultipartIdentifier(identifier) + } + } + val parts = astBuilder.extractIdentifierParts(ctx) + if (parts.size > 1) { + throw new ParseException( + errorClass = "IDENTIFIER_TOO_MANY_NAME_PARTS", + messageParameters = Map("identifier" -> parts.map(part => s"`$part`").mkString("."), + "limit" -> "1"), + ctx) + } + parts.head + } + private def checkLabels( beginLabelCtx: Option[BeginLabelContext], - endLabelCtx: Option[EndLabelContext]) : Unit = { + endLabelCtx: Option[EndLabelContext]): Unit = { + // Check label matching and other constraints. (beginLabelCtx, endLabelCtx) match { // Throw an error if labels do not match. - case (Some(bl: BeginLabelContext), Some(el: EndLabelContext)) - if bl.multipartIdentifier().getText.toLowerCase(Locale.ROOT) != - el.multipartIdentifier().getText.toLowerCase(Locale.ROOT) => - withOrigin(bl) { - throw SqlScriptingErrors.labelsMismatch( - CurrentOrigin.get, - bl.multipartIdentifier().getText, - el.multipartIdentifier().getText) - } - // Throw an error if label is qualified. - case (Some(bl: BeginLabelContext), _) - if bl.multipartIdentifier().parts.size() > 1 => - withOrigin(bl) { - throw SqlScriptingErrors.labelCannotBeQualified( - CurrentOrigin.get, - bl.multipartIdentifier().getText.toLowerCase(Locale.ROOT) - ) + case (Some(bl: BeginLabelContext), Some(el: EndLabelContext)) => + val beginLabel = getLabelText(bl.strictIdentifier()).toLowerCase(Locale.ROOT) + val endLabel = getLabelText(el.strictIdentifier()).toLowerCase(Locale.ROOT) + if (beginLabel != endLabel) { + withOrigin(bl) { + throw SqlScriptingErrors.labelsMismatch( + CurrentOrigin.get, + getLabelText(bl.strictIdentifier()), + getLabelText(el.strictIdentifier())) + } } // Throw an error if end label exists without begin label. case (None, Some(el: EndLabelContext)) => withOrigin(el) { throw SqlScriptingErrors.endLabelWithoutBeginLabel( - CurrentOrigin.get, el.multipartIdentifier().getText) + CurrentOrigin.get, + getLabelText(el.strictIdentifier())) } case _ => } @@ -314,7 +329,7 @@ class SqlScriptingLabelContext { /** Check if the label is defined. */ private def isLabelDefined(beginLabelCtx: Option[BeginLabelContext]): Boolean = { - beginLabelCtx.map(_.multipartIdentifier().getText).isDefined + beginLabelCtx.isDefined } /** @@ -322,13 +337,13 @@ class SqlScriptingLabelContext { * If the identifier is contained within seenLabels, raise an exception. */ private def assertIdentifierNotInSeenLabels( - identifierCtx: Option[MultipartIdentifierContext]): Unit = { + identifierCtx: Option[StrictIdentifierContext]): Unit = { identifierCtx.foreach { ctx => - val identifierName = ctx.getText - if (seenLabels.contains(identifierName.toLowerCase(Locale.ROOT))) { + val identifierName = getLabelText(ctx).toLowerCase(Locale.ROOT) + if (seenLabels.contains(identifierName)) { withOrigin(ctx) { throw SqlScriptingErrors - .duplicateLabels(CurrentOrigin.get, identifierName.toLowerCase(Locale.ROOT)) + .duplicateLabels(CurrentOrigin.get, identifierName) } } } @@ -348,7 +363,7 @@ class SqlScriptingLabelContext { // Get label text and add it to seenLabels. val labelText = if (isLabelDefined(beginLabelCtx)) { - val txt = beginLabelCtx.get.multipartIdentifier().getText.toLowerCase(Locale.ROOT) + val txt = getLabelText(beginLabelCtx.get.strictIdentifier()).toLowerCase(Locale.ROOT) if (seenLabels.contains(txt)) { withOrigin(beginLabelCtx.get) { throw SqlScriptingErrors.duplicateLabels(CurrentOrigin.get, txt) @@ -374,18 +389,18 @@ class SqlScriptingLabelContext { */ def exitLabeledScope(beginLabelCtx: Option[BeginLabelContext]): Unit = { if (isLabelDefined(beginLabelCtx)) { - seenLabels.remove(beginLabelCtx.get.multipartIdentifier().getText.toLowerCase(Locale.ROOT)) + seenLabels.remove(getLabelText(beginLabelCtx.get.strictIdentifier()).toLowerCase(Locale.ROOT)) } } /** * Enter a for loop scope. - * If the for loop variable is defined, it will be asserted to not be inside seenLabels; + * If the for loop variable is defined, it will be asserted to not be inside seenLabels. * Then, if the for loop variable is defined, it will be added to seenLabels. */ - def enterForScope(identifierCtx: Option[MultipartIdentifierContext]): Unit = { + def enterForScope(identifierCtx: Option[StrictIdentifierContext]): Unit = { identifierCtx.foreach { ctx => - val identifierName = ctx.getText + val identifierName = getLabelText(ctx) assertIdentifierNotInSeenLabels(identifierCtx) seenLabels.add(identifierName.toLowerCase(Locale.ROOT)) @@ -403,9 +418,9 @@ class SqlScriptingLabelContext { * Exit a for loop scope. * If the for loop variable is defined, it will be removed from seenLabels. */ - def exitForScope(identifierCtx: Option[MultipartIdentifierContext]): Unit = { + def exitForScope(identifierCtx: Option[StrictIdentifierContext]): Unit = { identifierCtx.foreach { ctx => - val identifierName = ctx.getText + val identifierName = getLabelText(ctx) seenLabels.remove(identifierName.toLowerCase(Locale.ROOT)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SubstituteParamsParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SubstituteParamsParser.scala index 54c8c2ec089f..9beead0e6487 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SubstituteParamsParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SubstituteParamsParser.scala @@ -186,6 +186,8 @@ class SubstituteParamsParser extends Logging { /** * Apply a list of substitutions to the SQL text. + * Inserts a space separator when a parameter is immediately preceded by a quote + * to avoid back-to-back quotes after substitution. */ private def applySubstitutions(sqlText: String, substitutions: List[Substitution]): String = { // Sort substitutions by start position in reverse order to avoid offset issues @@ -193,9 +195,18 @@ class SubstituteParamsParser extends Logging { var result = sqlText sortedSubstitutions.foreach { substitution => - result = result.substring(0, substitution.start) + - substitution.replacement + - result.substring(substitution.end) + val prefix = result.substring(0, substitution.start) + val replacement = substitution.replacement + val suffix = result.substring(substitution.end) + + // Check if replacement is immediately preceded by a quote and doesn't already + // start with whitespace + val needsSpace = substitution.start > 0 && + (result(substitution.start - 1) == '\'' || result(substitution.start - 1) == '"') && + replacement.nonEmpty && !replacement(0).isWhitespace + + val space = if (needsSpace) " " else "" + result = s"$prefix$space$replacement$suffix" } result } @@ -211,4 +222,3 @@ object SubstituteParamsParser { positionalParams: List[String] = List.empty): (String, Int, PositionMapper) = instance.substitute(sqlText, namedParams, positionalParams) } - diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 9847dbd6b197..ece309e3b856 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -2247,13 +2247,17 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat def identifierTooManyNamePartsError(originalIdentifier: String): Throwable = { new AnalysisException( errorClass = "IDENTIFIER_TOO_MANY_NAME_PARTS", - messageParameters = Map("identifier" -> toSQLId(originalIdentifier))) + messageParameters = Map( + "identifier" -> toSQLId(originalIdentifier), + "limit" -> "2")) } def identifierTooManyNamePartsError(names: Seq[String]): Throwable = { new AnalysisException( errorClass = "IDENTIFIER_TOO_MANY_NAME_PARTS", - messageParameters = Map("identifier" -> toSQLId(names))) + messageParameters = Map( + "identifier" -> toSQLId(names), + "limit" -> "2")) } def emptyMultipartIdentifierError(): Throwable = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index 1f7d2a149a7b..27aba1f7f2df 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -1120,10 +1120,12 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE cause = e) } - def ddlUnsupportedTemporarilyError(ddl: String): SparkUnsupportedOperationException = { + def ddlUnsupportedTemporarilyError( + ddl: String, + tableName: String): SparkUnsupportedOperationException = { new SparkUnsupportedOperationException( - errorClass = "_LEGACY_ERROR_TEMP_2096", - messageParameters = Map("ddl" -> ddl)) + errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + messageParameters = Map("tableName" -> toSQLId(tableName), "operation" -> ddl)) } def executeBroadcastTimeoutError(timeout: Long, ex: Option[TimeoutException]): Throwable = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 38e823a96cbf..0ef1496cf9dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -5022,6 +5022,17 @@ object SQLConf { .booleanConf .createWithDefault(false) + val LEGACY_IDENTIFIER_CLAUSE_ONLY = + buildConf("spark.sql.legacy.identifierClause") + .internal() + .doc("When set to false, IDENTIFIER('literal') is resolved to an identifier at parse time " + + "anywhere identifiers can occur. When set to true, only the legacy " + + " IDENTIFIER(constantExpr) clause is allowed, which evaluates the expression at analysis " + + " and is limited to a narrow subset of scenarios.") + .version("4.1.0") + .booleanConf + .createWithDefault(false) + val LEGACY_ALLOW_NEGATIVE_SCALE_OF_DECIMAL_ENABLED = buildConf("spark.sql.legacy.allowNegativeScaleOfDecimal") .internal() @@ -7701,6 +7712,9 @@ class SQLConf extends Serializable with Logging with SqlApiConf { override def legacyParameterSubstitutionConstantsOnly: Boolean = getConf(SQLConf.LEGACY_PARAMETER_SUBSTITUTION_CONSTANTS_ONLY) + override def legacyIdentifierClauseOnly: Boolean = + getConf(SQLConf.LEGACY_IDENTIFIER_CLAUSE_ONLY) + def streamStatePollingInterval: Long = getConf(SQLConf.PIPELINES_STREAM_STATE_POLLING_INTERVAL) def watchdogMinRetryTimeInSeconds: Long = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala index 298329db1ee3..8c4c9044248d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala @@ -2273,11 +2273,11 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END; |END""".stripMargin checkError( - exception = intercept[SqlScriptingException] { + exception = intercept[ParseException] { parsePlan(sqlScriptText) }, - condition = "INVALID_LABEL_USAGE.QUALIFIED_LABEL_NAME", - parameters = Map("labelName" -> "PART1.PART2")) + condition = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'.'", "hint" -> "")) } test("qualified label name: label cannot be qualified + end label") { @@ -2288,11 +2288,11 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END part1.part2; |END""".stripMargin checkError( - exception = intercept[SqlScriptingException] { + exception = intercept[ParseException] { parsePlan(sqlScriptText) }, - condition = "INVALID_LABEL_USAGE.QUALIFIED_LABEL_NAME", - parameters = Map("labelName" -> "PART1.PART2")) + condition = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'.'", "hint" -> "")) } test("unique label names: nested labeled scope statements") { @@ -2785,13 +2785,13 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |BEGIN | DECLARE TEST.CONDITION CONDITION FOR SQLSTATE '12345'; |END""".stripMargin - val exception = intercept[SqlScriptingException] { + val exception = intercept[ParseException] { parsePlan(sqlScriptText) } checkError( exception = exception, - condition = "INVALID_ERROR_CONDITION_DECLARATION.QUALIFIED_CONDITION_NAME", - parameters = Map("conditionName" -> "TEST.CONDITION")) + condition = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'FOR'", "hint" -> ": missing ';'")) assert(exception.origin.line.contains(3)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala index f7876d9a023b..c47e84541621 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala @@ -45,11 +45,11 @@ import org.apache.spark.sql.catalyst.analysis.{GeneralParameterizedQuery, NamePa import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression, Literal} import org.apache.spark.sql.catalyst.parser.{HybridParameterContext, NamedParameterContext, ParserInterface, PositionalParameterContext} -import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, LocalRelation, OneRowRelation, Project, Range} +import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, LocalRelation, LogicalPlan, OneRowRelation, Project, Range} import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.classic.SparkSession.applyAndLoadExtensions -import org.apache.spark.sql.errors.SqlScriptingErrors +import org.apache.spark.sql.errors.{QueryCompilationErrors, SqlScriptingErrors} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.ExternalCommandExecutor import org.apache.spark.sql.execution.datasources.LogicalRelation @@ -501,26 +501,31 @@ class SparkSession private( private[sql] def sql(sqlText: String, args: Array[_], tracker: QueryPlanningTracker): DataFrame = withActive { val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) { - val parsedPlan = if (args.nonEmpty) { - // Resolve and validate parameters first - val paramMap = args.zipWithIndex.map { case (arg, idx) => - s"_pos_$idx" -> lit(arg).expr - }.toMap - val resolvedParams = resolveAndValidateParameters(paramMap) + val parsedPlan = { + // Always parse with parameter context to detect unbound parameter markers. + // Even if args is empty, we need to detect and reject parameter markers in the SQL. + val (paramMap, resolvedParams) = if (args.nonEmpty) { + val pMap = args.zipWithIndex.map { case (arg, idx) => + s"_pos_$idx" -> lit(arg).expr + }.toMap + (pMap, resolveAndValidateParameters(pMap)) + } else { + (Map.empty[String, Expression], Map.empty[String, Expression]) + } + val paramContext = PositionalParameterContext(resolvedParams.values.toSeq) val parsed = sessionState.sqlParser.parsePlanWithParameters(sqlText, paramContext) + // Check for SQL scripting with positional parameters - if (parsed.isInstanceOf[CompoundBody]) { + if (parsed.isInstanceOf[CompoundBody] && args.nonEmpty) { throw SqlScriptingErrors.positionalParametersAreNotSupportedWithSqlScripting() } // In legacy mode, wrap with PosParameterizedQuery for analyzer binding - if (sessionState.conf.legacyParameterSubstitutionConstantsOnly) { + if (args.nonEmpty && sessionState.conf.legacyParameterSubstitutionConstantsOnly) { PosParameterizedQuery(parsed, paramContext.params) } else { parsed } - } else { - sessionState.sqlParser.parsePlan(sqlText) } parsedPlan } @@ -554,30 +559,29 @@ class SparkSession private( args: Map[String, Any], tracker: QueryPlanningTracker): DataFrame = withActive { - // Always set parameter context if we have actual parameters - if (args.nonEmpty) { - // Resolve and validate parameters first - val resolvedParams = resolveAndValidateParameters(args.transform((_, v) => lit(v).expr)) - val paramContext = NamedParameterContext(resolvedParams) - val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) { - val parsedPlan = sessionState.sqlParser.parsePlanWithParameters(sqlText, paramContext) - // In legacy mode, wrap the parsed plan with NameParameterizedQuery - // so that the BindParameters analyzer rule can bind the parameters - if (sessionState.conf.legacyParameterSubstitutionConstantsOnly) { - NameParameterizedQuery(parsedPlan, paramContext.params) - } else { - parsedPlan - } - } - - Dataset.ofRows(self, plan, tracker) + // Always parse with parameter context to detect unbound parameter markers. + // Even if args is empty, we need to detect and reject parameter markers in the SQL. + val resolvedParams = if (args.nonEmpty) { + resolveAndValidateParameters(args.transform((_, v) => lit(v).expr)) } else { - // No parameters - parse normally without parameter context - val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) { - sessionState.sqlParser.parsePlan(sqlText) + Map.empty[String, Expression] + } + val paramContext = NamedParameterContext(resolvedParams) + val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) { + val parsedPlan = sessionState.sqlParser.parsePlanWithParameters(sqlText, paramContext) + val queryPlan = parsedPlan match { + case compoundBody: CompoundBody => compoundBody + case logicalPlan: LogicalPlan => + // In legacy mode, wrap with NameParameterizedQuery for analyzer binding + if (args.nonEmpty && sessionState.conf.legacyParameterSubstitutionConstantsOnly) { + NameParameterizedQuery(logicalPlan, paramContext.params) + } else { + logicalPlan + } } - Dataset.ofRows(self, plan, tracker) + queryPlan } + Dataset.ofRows(self, plan, tracker) } /** @inheritdoc */ @@ -610,6 +614,8 @@ class SparkSession private( tracker: QueryPlanningTracker): DataFrame = withActive { val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) { + // Always parse with parameter context to detect unbound parameter markers. + // Even if args is empty, we need to detect and reject parameter markers in the SQL. val parsedPlan = if (args.nonEmpty) { // Resolve and validate parameter arguments val paramMap = args.zipWithIndex.map { case (arg, idx) => @@ -643,11 +649,6 @@ class SparkSession private( val parsed = sessionState.sqlParser.parsePlanWithParameters(sqlText, paramContext) - // Check for SQL scripting with positional parameters - if (parsed.isInstanceOf[CompoundBody] && paramNames.isEmpty) { - throw SqlScriptingErrors.positionalParametersAreNotSupportedWithSqlScripting() - } - // In legacy mode, wrap with GeneralParameterizedQuery for analyzer binding if (sessionState.conf.legacyParameterSubstitutionConstantsOnly) { GeneralParameterizedQuery( @@ -659,8 +660,16 @@ class SparkSession private( parsed } } else { - sessionState.sqlParser.parsePlan(sqlText) + // No arguments provided, but still need to detect parameter markers + val paramContext = HybridParameterContext(Seq.empty, Seq.empty) + sessionState.sqlParser.parsePlanWithParameters(sqlText, paramContext) } + + // Check for SQL scripts in EXECUTE IMMEDIATE (applies to both empty and non-empty args) + if (parsedPlan.isInstanceOf[CompoundBody]) { + throw QueryCompilationErrors.sqlScriptInExecuteImmediate(sqlText) + } + parsedPlan } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 58bffbed3e69..550c23e3e830 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -118,14 +118,22 @@ class SparkSqlParser extends AbstractSqlParser { // Step 2: Apply parameter substitution if a parameter context is provided. val (paramSubstituted, positionMapper, hasParameters) = parameterContext match { case Some(context) => + // Check if the context actually contains parameters + val contextHasParams = context match { + case NamedParameterContext(params) => params.nonEmpty + case PositionalParameterContext(params) => params.nonEmpty + case HybridParameterContext(args, _) => args.nonEmpty + } if (SQLConf.get.legacyParameterSubstitutionConstantsOnly) { // Legacy mode: Parameters are detected but substitution is deferred to analysis phase. - (variableSubstituted, PositionMapper.identity(variableSubstituted), true) + // Only set hasParameters if the context actually contains parameters. + (variableSubstituted, PositionMapper.identity(variableSubstituted), contextHasParams) } else { // Modern mode: Perform parameter substitution during parsing. val (substituted, mapper) = ParameterHandler.substituteParameters(variableSubstituted, context) - (substituted, mapper, true) + // Only set hasParameters if the context actually contains parameters. + (substituted, mapper, contextHasParams) } case None => // No parameter context provided; skip parameter substitution. @@ -629,7 +637,8 @@ class SparkSqlAstBuilder extends AstBuilder { val userSpecifiedColumns = Option(ctx.identifierCommentList).toSeq.flatMap { icl => icl.identifierComment.asScala.map { ic => - ic.identifier.getText -> Option(ic.commentSpec()).map(visitCommentSpec) + // Use getIdentifierText to handle both regular identifiers and IDENTIFIER('literal') + getIdentifierText(ic.identifier) -> Option(ic.commentSpec()).map(visitCommentSpec) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 3c36d3e2d417..5efad83bcba7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -23,6 +23,7 @@ import org.apache.spark.{SparkException, SparkUnsupportedOperationException} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{execution, AnalysisException} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, NamedRelation} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, JoinSelectionHelper, NormalizeFloatingNumbers} @@ -31,12 +32,14 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.streaming.{InternalOutputModes, StreamingRelationV2} import org.apache.spark.sql.catalyst.types.DataTypeUtils +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.{SparkStrategy => Strategy} import org.apache.spark.sql.execution.aggregate.AggUtils import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec} import org.apache.spark.sql.execution.command._ -import org.apache.spark.sql.execution.datasources.{WriteFiles, WriteFilesExec} +import org.apache.spark.sql.execution.datasources.{LogicalRelation, WriteFiles, WriteFilesExec} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.exchange.{REBALANCE_PARTITIONS_BY_COL, REBALANCE_PARTITIONS_BY_NONE, REPARTITION_BY_COL, REPARTITION_BY_NUM, ShuffleExchangeExec} import org.apache.spark.sql.execution.python._ import org.apache.spark.sql.execution.python.streaming.{FlatMapGroupsInPandasWithStateExec, TransformWithStateInPySparkExec} @@ -1091,10 +1094,12 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case r: LogicalRDD => RDDScanExec(r.output, r.rdd, "ExistingRDD", r.outputPartitioning, r.outputOrdering, r.stream) :: Nil - case _: UpdateTable => - throw QueryExecutionErrors.ddlUnsupportedTemporarilyError("UPDATE TABLE") - case _: MergeIntoTable => - throw QueryExecutionErrors.ddlUnsupportedTemporarilyError("MERGE INTO TABLE") + case u: UpdateTable => + val tableName = extractTableNameForError(u.table) + throw QueryExecutionErrors.ddlUnsupportedTemporarilyError("UPDATE TABLE", tableName) + case m: MergeIntoTable => + val tableName = extractTableNameForError(m.targetTable) + throw QueryExecutionErrors.ddlUnsupportedTemporarilyError("MERGE INTO TABLE", tableName) case logical.CollectMetrics(name, metrics, child, _) => execution.CollectMetricsExec(name, metrics, planLater(child)) :: Nil case WriteFiles(child, fileFormat, partitionColumns, bucket, options, staticPartitions) => @@ -1105,4 +1110,22 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case _ => Nil } } + + /** + * Extracts a user-friendly table name from a logical plan for error messages. + */ + private def extractTableNameForError(table: LogicalPlan): String = { + val unwrapped = EliminateSubqueryAliases(table) + unwrapped match { + // Check specific types before NamedRelation since they extend it + case DataSourceV2Relation(_, _, catalog, Some(ident), _, _) => + (catalog.map(_.name()).toSeq ++ ident.asMultipartIdentifier).mkString(".") + case LogicalRelation(_, _, Some(catalogTable), _, _) => + catalogTable.identifier.unquotedString + case r: NamedRelation => + r.name + case _ => + "unknown" + } + } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out index 1271f730d1e5..c874945badb1 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out @@ -1224,3 +1224,43 @@ EXECUTE IMMEDIATE 'SELECT typeof(:p) as type, :p as val' USING MAP(1, 'one', 2, -- !query analysis Project [typeof(map(1, one, 2, two)) AS type#x, map(1, one, 2, two) AS val#x] +- OneRowRelation + + +-- !query +EXECUTE IMMEDIATE 'SELECT :param' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNBOUND_SQL_PARAMETER", + "sqlState" : "42P02", + "messageParameters" : { + "name" : "param" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 13, + "fragment" : ":param" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT ?' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNBOUND_SQL_PARAMETER", + "sqlState" : "42P02", + "messageParameters" : { + "name" : "_7" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 8, + "fragment" : "?" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out new file mode 100644 index 000000000000..17fcc9b47729 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause-legacy.sql.out @@ -0,0 +1,2431 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SET hivevar:colname = 'c' +-- !query analysis +SetCommand (hivevar:colname,Some('c')) + + +-- !query +SELECT IDENTIFIER(${colname} || '_1') FROM VALUES(1) AS T(c_1) +-- !query analysis +Project [c_1#x] ++- SubqueryAlias T + +- LocalRelation [c_1#x] + + +-- !query +SELECT IDENTIFIER('c1') FROM VALUES(1) AS T(c1) +-- !query analysis +Project [c1#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +SELECT IDENTIFIER('t.c1') FROM VALUES(1) AS T(c1) +-- !query analysis +Project [c1#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +SELECT IDENTIFIER('`t`.c1') FROM VALUES(1) AS T(c1) +-- !query analysis +Project [c1#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +SELECT IDENTIFIER('`c 1`') FROM VALUES(1) AS T(`c 1`) +-- !query analysis +Project [c 1#x] ++- SubqueryAlias T + +- LocalRelation [c 1#x] + + +-- !query +SELECT IDENTIFIER('``') FROM VALUES(1) AS T(``) +-- !query analysis +Project [#x] ++- SubqueryAlias T + +- LocalRelation [#x] + + +-- !query +SELECT IDENTIFIER('c' || '1') FROM VALUES(1) AS T(c1) +-- !query analysis +Project [c1#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +CREATE SCHEMA IF NOT EXISTS s +-- !query analysis +CreateNamespace true ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [s] + + +-- !query +CREATE TABLE s.tab(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`s`.`tab`, false + + +-- !query +USE SCHEMA s +-- !query analysis +SetNamespaceCommand [s] + + +-- !query +INSERT INTO IDENTIFIER('ta' || 'b') VALUES(1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/s.db/tab, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/s.db/tab], Append, `spark_catalog`.`s`.`tab`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/s.db/tab), [c1] ++- Project [col1#x AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +DELETE FROM IDENTIFIER('ta' || 'b') WHERE 1=0 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "DELETE", + "tableName" : "`spark_catalog`.`s`.`tab`" + } +} + + +-- !query +UPDATE IDENTIFIER('ta' || 'b') SET c1 = 2 +-- !query analysis +org.apache.spark.SparkUnsupportedOperationException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "UPDATE TABLE", + "tableName" : "`spark_catalog`.`s`.`tab`" + } +} + + +-- !query +MERGE INTO IDENTIFIER('ta' || 'b') AS t USING IDENTIFIER('ta' || 'b') AS s ON s.c1 = t.c1 + WHEN MATCHED THEN UPDATE SET c1 = 3 +-- !query analysis +org.apache.spark.SparkUnsupportedOperationException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "MERGE INTO TABLE", + "tableName" : "`spark_catalog`.`s`.`tab`" + } +} + + +-- !query +SELECT * FROM IDENTIFIER('tab') +-- !query analysis +Project [c1#x] ++- SubqueryAlias spark_catalog.s.tab + +- Relation spark_catalog.s.tab[c1#x] csv + + +-- !query +SELECT * FROM IDENTIFIER('s.tab') +-- !query analysis +Project [c1#x] ++- SubqueryAlias spark_catalog.s.tab + +- Relation spark_catalog.s.tab[c1#x] csv + + +-- !query +SELECT * FROM IDENTIFIER('`s`.`tab`') +-- !query analysis +Project [c1#x] ++- SubqueryAlias spark_catalog.s.tab + +- Relation spark_catalog.s.tab[c1#x] csv + + +-- !query +SELECT * FROM IDENTIFIER('t' || 'a' || 'b') +-- !query analysis +Project [c1#x] ++- SubqueryAlias spark_catalog.s.tab + +- Relation spark_catalog.s.tab[c1#x] csv + + +-- !query +USE SCHEMA default +-- !query analysis +SetNamespaceCommand [default] + + +-- !query +DROP TABLE s.tab +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), s.tab + + +-- !query +DROP SCHEMA s +-- !query analysis +DropNamespace false, false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [s] + + +-- !query +SELECT IDENTIFIER('COAL' || 'ESCE')(NULL, 1) +-- !query analysis +Project [coalesce(cast(null as int), 1) AS coalesce(NULL, 1)#x] ++- OneRowRelation + + +-- !query +SELECT IDENTIFIER('abs')(c1) FROM VALUES(-1) AS T(c1) +-- !query analysis +Project [abs(c1#x) AS abs(c1)#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +SELECT * FROM IDENTIFIER('ra' || 'nge')(0, 1) +-- !query analysis +Project [id#xL] ++- Range (0, 1, step=1) + + +-- !query +CREATE TABLE IDENTIFIER('tab')(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tab`, false + + +-- !query +DROP TABLE IF EXISTS IDENTIFIER('ta' || 'b') +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tab + + +-- !query +CREATE SCHEMA identifier_clauses +-- !query analysis +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [identifier_clauses] + + +-- !query +USE identifier_clauses +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [identifier_clauses] + + +-- !query +CREATE TABLE IDENTIFIER('ta' || 'b')(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clauses`.`tab`, false + + +-- !query +DROP TABLE IF EXISTS IDENTIFIER('identifier_clauses.' || 'tab') +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clauses.tab + + +-- !query +CREATE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clauses`.`tab`, false + + +-- !query +REPLACE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "REPLACE TABLE", + "tableName" : "`spark_catalog`.`identifier_clauses`.`tab`" + } +} + + +-- !query +CACHE TABLE IDENTIFIER('ta' || 'b') +-- !query analysis +CacheTable [tab], false, true + +- SubqueryAlias spark_catalog.identifier_clauses.tab + +- Relation spark_catalog.identifier_clauses.tab[c1#x] csv + + +-- !query +UNCACHE TABLE IDENTIFIER('ta' || 'b') +-- !query analysis +UncacheTable false, true + +- SubqueryAlias spark_catalog.identifier_clauses.tab + +- Relation spark_catalog.identifier_clauses.tab[c1#x] csv + + +-- !query +DROP TABLE IF EXISTS IDENTIFIER('ta' || 'b') +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clauses.tab + + +-- !query +USE default +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [default] + + +-- !query +DROP SCHEMA identifier_clauses +-- !query analysis +DropNamespace false, false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [identifier_clauses] + + +-- !query +CREATE TABLE tab(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tab`, false + + +-- !query +INSERT INTO tab VALUES (1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tab, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/tab], Append, `spark_catalog`.`default`.`tab`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tab), [c1] ++- Project [col1#x AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT c1 FROM tab +-- !query analysis +Project [c1#x] ++- SubqueryAlias spark_catalog.default.tab + +- Relation spark_catalog.default.tab[c1#x] csv + + +-- !query +DESCRIBE IDENTIFIER('ta' || 'b') +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`tab`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +ANALYZE TABLE IDENTIFIER('ta' || 'b') COMPUTE STATISTICS +-- !query analysis +AnalyzeTableCommand `spark_catalog`.`default`.`tab`, false + + +-- !query +ALTER TABLE IDENTIFIER('ta' || 'b') ADD COLUMN c2 INT +-- !query analysis +AlterTableAddColumnsCommand `spark_catalog`.`default`.`tab`, [StructField(c2,IntegerType,true)] + + +-- !query +SHOW TBLPROPERTIES IDENTIFIER('ta' || 'b') +-- !query analysis +ShowTableProperties [key#x, value#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.tab, V1Table(default.tab), [c1#x, c2#x] + + +-- !query +SHOW COLUMNS FROM IDENTIFIER('ta' || 'b') +-- !query analysis +ShowColumnsCommand `spark_catalog`.`default`.`tab`, [col_name#x] + + +-- !query +COMMENT ON TABLE IDENTIFIER('ta' || 'b') IS 'hello' +-- !query analysis +CommentOnTable hello ++- ResolvedTable V2SessionCatalog(spark_catalog), default.tab, V1Table(default.tab), [c1#x, c2#x] + + +-- !query +REFRESH TABLE IDENTIFIER('ta' || 'b') +-- !query analysis +RefreshTableCommand `spark_catalog`.`default`.`tab` + + +-- !query +REPAIR TABLE IDENTIFIER('ta' || 'b') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_PARTITIONED_TABLE", + "sqlState" : "42809", + "messageParameters" : { + "operation" : "MSCK REPAIR TABLE", + "tableIdentWithDB" : "`spark_catalog`.`default`.`tab`" + } +} + + +-- !query +TRUNCATE TABLE IDENTIFIER('ta' || 'b') +-- !query analysis +TruncateTableCommand `spark_catalog`.`default`.`tab` + + +-- !query +DROP TABLE IF EXISTS tab +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tab + + +-- !query +CREATE OR REPLACE VIEW IDENTIFIER('v')(c1) AS VALUES(1) +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`v`, [(c1,None)], VALUES(1), false, true, PersistedView, COMPENSATION, true + +- LocalRelation [col1#x] + + +-- !query +SELECT * FROM v +-- !query analysis +Project [c1#x] ++- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x]) + +- Project [cast(col1#x as int) AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +ALTER VIEW IDENTIFIER('v') AS VALUES(2) +-- !query analysis +AlterViewAsCommand `spark_catalog`.`default`.`v`, VALUES(2), true + +- LocalRelation [col1#x] + + +-- !query +DROP VIEW IDENTIFIER('v') +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`v`, false, true, false + + +-- !query +CREATE TEMPORARY VIEW IDENTIFIER('v')(c1) AS VALUES(1) +-- !query analysis +CreateViewCommand `v`, [(c1,None)], VALUES(1), false, false, LocalTempView, UNSUPPORTED, true + +- LocalRelation [col1#x] + + +-- !query +DROP VIEW IDENTIFIER('v') +-- !query analysis +DropTempViewCommand v + + +-- !query +CREATE SCHEMA IDENTIFIER('id' || 'ent') +-- !query analysis +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +ALTER SCHEMA IDENTIFIER('id' || 'ent') SET PROPERTIES (somekey = 'somevalue') +-- !query analysis +SetNamespaceProperties [somekey=somevalue] ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +ALTER SCHEMA IDENTIFIER('id' || 'ent') SET LOCATION 'someloc' +-- !query analysis +SetNamespaceLocation someloc ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +COMMENT ON SCHEMA IDENTIFIER('id' || 'ent') IS 'some comment' +-- !query analysis +CommentOnNamespace some comment ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +DESCRIBE SCHEMA IDENTIFIER('id' || 'ent') +-- !query analysis +DescribeNamespace false, [info_name#x, info_value#x] ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +SHOW TABLES IN IDENTIFIER('id' || 'ent') +-- !query analysis +ShowTables [namespace#x, tableName#x, isTemporary#x] ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +SHOW TABLE EXTENDED IN IDENTIFIER('id' || 'ent') LIKE 'hello' +-- !query analysis +ShowTablesCommand ident, hello, [namespace#x, tableName#x, isTemporary#x, information#x], true + + +-- !query +USE IDENTIFIER('id' || 'ent') +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +SHOW CURRENT SCHEMA +-- !query analysis +ShowCurrentNamespaceCommand + + +-- !query +USE SCHEMA IDENTIFIER('id' || 'ent') +-- !query analysis +SetNamespaceCommand [ident] + + +-- !query +USE SCHEMA default +-- !query analysis +SetNamespaceCommand [default] + + +-- !query +DROP SCHEMA IDENTIFIER('id' || 'ent') +-- !query analysis +DropNamespace false, false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +CREATE SCHEMA ident +-- !query analysis +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +CREATE FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query analysis +CreateFunctionCommand spark_catalog.ident.myDoubleAvg, test.org.apache.spark.sql.MyDoubleAvg, false, false, false + + +-- !query +DESCRIBE FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') +-- !query analysis +DescribeFunctionCommand org.apache.spark.sql.catalyst.expressions.ExpressionInfo@xxxxxxxx, false + + +-- !query +REFRESH FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') +-- !query analysis +RefreshFunctionCommand ident, mydoubleavg + + +-- !query +DROP FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') +-- !query analysis +DropFunctionCommand spark_catalog.ident.mydoubleavg, false, false + + +-- !query +DROP SCHEMA ident +-- !query analysis +DropNamespace false, false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [ident] + + +-- !query +CREATE TEMPORARY FUNCTION IDENTIFIER('my' || 'DoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query analysis +CreateFunctionCommand myDoubleAvg, test.org.apache.spark.sql.MyDoubleAvg, true, false, false + + +-- !query +DROP TEMPORARY FUNCTION IDENTIFIER('my' || 'DoubleAvg') +-- !query analysis +DropFunctionCommand myDoubleAvg, false, true + + +-- !query +DECLARE var = 'sometable' +-- !query analysis +CreateVariable defaultvalueexpression(sometable, 'sometable'), false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var + + +-- !query +CREATE TABLE IDENTIFIER(var)(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`sometable`, false + + +-- !query +SET VAR var = 'c1' +-- !query analysis +SetVariable [variablereference(system.session.var='sometable')] ++- Project [c1 AS var#x] + +- OneRowRelation + + +-- !query +SELECT IDENTIFIER(var) FROM VALUES(1) AS T(c1) +-- !query analysis +Project [c1#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +SET VAR var = 'some' +-- !query analysis +SetVariable [variablereference(system.session.var='c1')] ++- Project [some AS var#x] + +- OneRowRelation + + +-- !query +DROP TABLE IDENTIFIER(var || 'table') +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.sometable + + +-- !query +SELECT IDENTIFIER('c 1') FROM VALUES(1) AS T(`c 1`) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'1'", + "hint" : ": extra input '1'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "IDENTIFIER('c 1')" + } ] +} + + +-- !query +SELECT IDENTIFIER('') FROM VALUES(1) AS T(``) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_EMPTY_STATEMENT", + "sqlState" : "42617", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "IDENTIFIER('')" + } ] +} + + +-- !query +VALUES(IDENTIFIER(CAST(NULL AS STRING))) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NULL", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "CAST(NULL AS STRING)", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 38, + "fragment" : "CAST(NULL AS STRING)" + } ] +} + + +-- !query +VALUES(IDENTIFIER(1)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.WRONG_TYPE", + "sqlState" : "42601", + "messageParameters" : { + "dataType" : "int", + "expr" : "1", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 19, + "fragment" : "1" + } ] +} + + +-- !query +VALUES(IDENTIFIER(SUBSTR('HELLO', 1, RAND() + 1))) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "substr('HELLO', 1, CAST((rand() + CAST(1 AS DOUBLE)) AS INT))", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 48, + "fragment" : "SUBSTR('HELLO', 1, RAND() + 1)" + } ] +} + + +-- !query +SELECT `IDENTIFIER`('abs')(c1) FROM VALUES(-1) AS T(c1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`IDENTIFIER`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "`IDENTIFIER`('abs')" + } ] +} + + +-- !query +CREATE TABLE IDENTIFIER(1)(c1 INT) USING csv +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.WRONG_TYPE", + "sqlState" : "42601", + "messageParameters" : { + "dataType" : "int", + "expr" : "1", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 25, + "fragment" : "1" + } ] +} + + +-- !query +CREATE TABLE IDENTIFIER('a.b.c')(c1 INT) USING csv +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +CREATE VIEW IDENTIFIER('a.b.c')(c1) AS VALUES(1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +DROP TABLE IDENTIFIER('a.b.c') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +DROP VIEW IDENTIFIER('a.b.c') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +COMMENT ON TABLE IDENTIFIER('a.b.c.d') IS 'hello' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`.`c`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +VALUES(IDENTIFIER(1)()) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.WRONG_TYPE", + "sqlState" : "42601", + "messageParameters" : { + "dataType" : "int", + "expr" : "1", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 19, + "fragment" : "1" + } ] +} + + +-- !query +VALUES(IDENTIFIER('a.b.c.d')()) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", + "sqlState" : "42601", + "messageParameters" : { + "identifier" : "`a`.`b`.`c`.`d`", + "limit" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "IDENTIFIER('a.b.c.d')()" + } ] +} + + +-- !query +CREATE TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX.CREATE_TEMP_FUNC_WITH_DATABASE", + "sqlState" : "42000", + "messageParameters" : { + "database" : "`default`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 108, + "fragment" : "CREATE TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg'" + } ] +} + + +-- !query +DROP TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX.MULTI_PART_NAME", + "sqlState" : "42000", + "messageParameters" : { + "name" : "`default`.`myDoubleAvg`", + "statement" : "DROP TEMPORARY FUNCTION" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 63, + "fragment" : "DROP TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg')" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW IDENTIFIER('default.v')(c1) AS VALUES(1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "TEMP_VIEW_NAME_TOO_MANY_NAME_PARTS", + "sqlState" : "428EK", + "messageParameters" : { + "actualName" : "`default`.`v`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "CREATE TEMPORARY VIEW IDENTIFIER('default.v')(c1) AS VALUES(1)" + } ] +} + + +-- !query +create temporary view identifier('v1') as (select my_col from (values (1), (2), (1) as (my_col)) group by 1) +-- !query analysis +CreateViewCommand `v1`, (select my_col from (values (1), (2), (1) as (my_col)) group by 1), false, false, LocalTempView, UNSUPPORTED, true + +- Aggregate [my_col#x], [my_col#x] + +- SubqueryAlias __auto_generated_subquery_name + +- SubqueryAlias as + +- LocalRelation [my_col#x] + + +-- !query +cache table identifier('t1') as (select my_col from (values (1), (2), (1) as (my_col)) group by 1) +-- !query analysis +CacheTableAsSelect t1, (select my_col from (values (1), (2), (1) as (my_col)) group by 1), false, true + +- Aggregate [my_col#x], [my_col#x] + +- SubqueryAlias __auto_generated_subquery_name + +- SubqueryAlias as + +- LocalRelation [my_col#x] + + +-- !query +create table identifier('t2') using csv as (select my_col from (values (1), (2), (1) as (my_col)) group by 1) +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`t2`, ErrorIfExists, [my_col] + +- Aggregate [my_col#x], [my_col#x] + +- SubqueryAlias __auto_generated_subquery_name + +- SubqueryAlias as + +- LocalRelation [my_col#x] + + +-- !query +insert into identifier('t2') select my_col from (values (3) as (my_col)) group by 1 +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [my_col] ++- Project [my_col#x AS my_col#x] + +- Aggregate [my_col#x], [my_col#x] + +- SubqueryAlias __auto_generated_subquery_name + +- SubqueryAlias as + +- LocalRelation [my_col#x] + + +-- !query +drop view v1 +-- !query analysis +DropTempViewCommand v1 + + +-- !query +drop table t1 +-- !query analysis +DropTempViewCommand t1 + + +-- !query +drop table t2 +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t2 + + +-- !query +DECLARE agg = 'max' +-- !query analysis +CreateVariable defaultvalueexpression(max, 'max'), false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.agg + + +-- !query +DECLARE col = 'c1' +-- !query analysis +CreateVariable defaultvalueexpression(c1, 'c1'), false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.col + + +-- !query +DECLARE tab = 'T' +-- !query analysis +CreateVariable defaultvalueexpression(T, 'T'), false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.tab + + +-- !query +WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), + T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) +SELECT IDENTIFIER(agg)(IDENTIFIER(col)) FROM IDENTIFIER(tab) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias S +: +- Project [col1#x AS c1#x, col2#x AS c2#x] +: +- LocalRelation [col1#x, col2#x] +:- CTERelationDef xxxx, false +: +- SubqueryAlias T +: +- Project [col1#x AS c1#x, col2#x AS c2#x] +: +- LocalRelation [col1#x, col2#x] ++- Aggregate [max(c1#x) AS max(c1)#x] + +- SubqueryAlias T + +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, 2 + + +-- !query +WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), + T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) +SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('T') +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias S +: +- Project [col1#x AS c1#x, col2#x AS c2#x] +: +- LocalRelation [col1#x, col2#x] +:- CTERelationDef xxxx, false +: +- SubqueryAlias T +: +- Project [col1#x AS c1#x, col2#x AS c2#x] +: +- LocalRelation [col1#x, col2#x] ++- Aggregate [max(c1#x) AS max(c1)#x] + +- SubqueryAlias T + +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, 2 + + +-- !query +WITH ABC(c1, c2) AS (VALUES(1, 2), (2, 3)) +SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('A' || 'BC') +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias ABC +: +- Project [col1#x AS c1#x, col2#x AS c2#x] +: +- LocalRelation [col1#x, col2#x] ++- Aggregate [max(c1#x) AS max(c1)#x] + +- SubqueryAlias ABC + +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false, 2 + + +-- !query +SELECT row_number() OVER IDENTIFIER('x.win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''x.win''", + "hint" : "" + } +} + + +-- !query +SELECT T1.c1 FROM VALUES(1) AS T1(c1) JOIN VALUES(1) AS T2(c1) USING (IDENTIFIER('c1')) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT IDENTIFIER('t').c1 FROM VALUES(1) AS T(c1) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t`", + "proposal" : "`c1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "IDENTIFIER('t')" + } ] +} + + +-- !query +SELECT map('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''a''", + "hint" : "" + } +} + + +-- !query +SELECT named_struct('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''a''", + "hint" : "" + } +} + + +-- !query +SELECT * FROM s.IDENTIFIER('tab') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM IDENTIFIER('s').IDENTIFIER('tab') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'IDENTIFIER'", + "hint" : "" + } +} + + +-- !query +SELECT * FROM IDENTIFIER('s').tab +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'IDENTIFIER'", + "hint" : "" + } +} + + +-- !query +SELECT row_number() OVER IDENTIFIER('win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''win''", + "hint" : "" + } +} + + +-- !query +SELECT row_number() OVER win FROM VALUES(1) AS T(c1) WINDOW IDENTIFIER('win') AS (ORDER BY c1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing 'AS'" + } +} + + +-- !query +SELECT 1 AS IDENTIFIER('col1') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT my_table.* FROM VALUES (1, 2) AS IDENTIFIER('my_table')(IDENTIFIER('c1'), IDENTIFIER('c2')) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''my_table''", + "hint" : "" + } +} + + +-- !query +WITH identifier('v')(identifier('c1')) AS (VALUES(1)) (SELECT c1 FROM v) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''v''", + "hint" : "" + } +} + + +-- !query +CREATE OR REPLACE VIEW v(IDENTIFIER('c1')) AS VALUES(1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT c1 FROM v +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`v`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 16, + "fragment" : "v" + } ] +} + + +-- !query +CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +INSERT INTO tab(IDENTIFIER('c1')) VALUES(1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing ')'" + } +} + + +-- !query +SELECT c1 FROM tab +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 18, + "fragment" : "tab" + } ] +} + + +-- !query +ALTER TABLE IDENTIFIER('tab') RENAME COLUMN IDENTIFIER('c1') TO IDENTIFIER('col1') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT col1 FROM tab +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 18, + "stopIndex" : 20, + "fragment" : "tab" + } ] +} + + +-- !query +ALTER TABLE IDENTIFIER('tab') ADD COLUMN IDENTIFIER('c2') INT +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT c2 FROM tab +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 18, + "fragment" : "tab" + } ] +} + + +-- !query +ALTER TABLE IDENTIFIER('tab') DROP COLUMN IDENTIFIER('c2') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +ALTER TABLE IDENTIFIER('tab') RENAME TO IDENTIFIER('tab_renamed') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM tab_renamed +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`tab_renamed`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 25, + "fragment" : "tab_renamed" + } ] +} + + +-- !query +DROP TABLE IF EXISTS tab_renamed +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tab_renamed + + +-- !query +DROP TABLE IF EXISTS tab +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tab + + +-- !query +CREATE TABLE test_col_with_dot(IDENTIFIER('`col.with.dot`') INT) USING CSV +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE IF EXISTS test_col_with_dot +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.test_col_with_dot + + +-- !query +SELECT * FROM VALUES (1, 2) AS IDENTIFIER('schema.table')(c1, c2) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''schema.table''", + "hint" : "" + } +} + + +-- !query +SELECT 1 AS IDENTIFIER('col1.col2') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +CREATE SCHEMA identifier_clause_test_schema +-- !query analysis +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [identifier_clause_test_schema] + + +-- !query +USE identifier_clause_test_schema +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [identifier_clause_test_schema] + + +-- !query +CREATE TABLE test_show(c1 INT, c2 STRING) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_show`, false + + +-- !query +SHOW VIEWS IN IDENTIFIER('identifier_clause_test_schema') +-- !query analysis +ShowViewsCommand identifier_clause_test_schema, [namespace#x, viewName#x, isTemporary#x] + + +-- !query +SHOW PARTITIONS IDENTIFIER('test_show') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_PARTITION_OPERATION.PARTITION_SCHEMA_IS_EMPTY", + "sqlState" : "42601", + "messageParameters" : { + "name" : "`spark_catalog`.`identifier_clause_test_schema`.`test_show`" + } +} + + +-- !query +SHOW CREATE TABLE IDENTIFIER('test_show') +-- !query analysis +ShowCreateTable false, [createtab_stmt#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_show, V1Table(identifier_clause_test_schema.test_show), [c1#x, c2#x] + + +-- !query +DROP TABLE test_show +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_show + + +-- !query +CREATE TABLE test_desc(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, false + + +-- !query +DESCRIBE TABLE IDENTIFIER('test_desc') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESCRIBE FORMATTED IDENTIFIER('test_desc') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESCRIBE EXTENDED IDENTIFIER('test_desc') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC IDENTIFIER('test_desc') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DROP TABLE test_desc +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_desc + + +-- !query +CREATE TABLE test_comment(c1 INT, c2 STRING) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_comment`, false + + +-- !query +COMMENT ON TABLE IDENTIFIER('test_comment') IS 'table comment' +-- !query analysis +CommentOnTable table comment ++- ResolvedTable V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_comment, V1Table(identifier_clause_test_schema.test_comment), [c1#x, c2#x] + + +-- !query +ALTER TABLE test_comment ALTER COLUMN IDENTIFIER('c1') COMMENT 'column comment' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE test_comment +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_comment + + +-- !query +CREATE TABLE identifier_clause_test_schema.test_table(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table`, false + + +-- !query +ANALYZE TABLE IDENTIFIER('identifier_clause_test_schema.test_table') COMPUTE STATISTICS +-- !query analysis +AnalyzeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table`, false + + +-- !query +REFRESH TABLE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query analysis +RefreshTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table` + + +-- !query +DESCRIBE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +SHOW COLUMNS FROM IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query analysis +ShowColumnsCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table`, [col_name#x] + + +-- !query +DROP TABLE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_table + + +-- !query +DECLARE IDENTIFIER('my_var') = 'value' +-- !query analysis +CreateVariable defaultvalueexpression(value, 'value'), false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.my_var + + +-- !query +SET VAR IDENTIFIER('my_var') = 'new_value' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing EQ" + } +} + + +-- !query +SELECT IDENTIFIER('my_var') +-- !query analysis +Project [variablereference(system.session.my_var='value') AS variablereference(system.session.my_var='value')#x] ++- OneRowRelation + + +-- !query +DROP TEMPORARY VARIABLE IDENTIFIER('my_var') +-- !query analysis +DropVariable false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.my_var + + +-- !query +CREATE TEMPORARY FUNCTION test_udf(IDENTIFIER('param1') INT, IDENTIFIER('param2') STRING) +RETURNS INT +RETURN IDENTIFIER('param1') + length(IDENTIFIER('param2')) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT test_udf(5, 'hello') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`test_udf`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`identifier_clause_test_schema`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "test_udf(5, 'hello')" + } ] +} + + +-- !query +DROP TEMPORARY FUNCTION test_udf +-- !query analysis +org.apache.spark.sql.catalyst.analysis.NoSuchTempFunctionException +{ + "errorClass" : "ROUTINE_NOT_FOUND", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`test_udf`" + } +} + + +-- !query +CREATE TEMPORARY FUNCTION test_table_udf(IDENTIFIER('input_val') INT) +RETURNS TABLE(IDENTIFIER('col1') INT, IDENTIFIER('col2') STRING) +RETURN SELECT IDENTIFIER('input_val'), 'result' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM test_table_udf(42) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVABLE_TABLE_VALUED_FUNCTION", + "sqlState" : "42883", + "messageParameters" : { + "name" : "`test_table_udf`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 32, + "fragment" : "test_table_udf(42)" + } ] +} + + +-- !query +DROP TEMPORARY FUNCTION test_table_udf +-- !query analysis +org.apache.spark.sql.catalyst.analysis.NoSuchTempFunctionException +{ + "errorClass" : "ROUTINE_NOT_FOUND", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`test_table_udf`" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:tab \'b\').c1 FROM VALUES(1) AS tab(c1)' USING 'ta' AS tab +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_BASE_FIELD_TYPE", + "sqlState" : "42000", + "messageParameters" : { + "base" : "\"variablereference(system.session.tab='T')\"", + "other" : "\"STRING\"" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col1 ''.c2'') FROM VALUES(named_struct(''c2'', 42)) AS T(c1)' + USING 'c1' AS col1 +-- !query analysis +Project [c1#x.c2 AS c1.c2#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +CREATE TABLE integration_test(c1 INT, c2 STRING) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`integration_test`, false + + +-- !query +INSERT INTO integration_test VALUES (1, 'a'), (2, 'b') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`integration_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test), [c1, c2] ++- Project [col1#x AS c1#x, col2#x AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table +-- !query analysis +Sort [c1#x ASC NULLS FIRST, c2#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''1''), IDENTIFIER(:prefix ''2'') FROM integration_test ORDER BY ALL' + USING 'c' AS prefix +-- !query analysis +Sort [c1#x ASC NULLS FIRST, c2#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM integration_test WHERE IDENTIFIER(:col) = :val' + USING 'c1' AS col, 1 AS val +-- !query analysis +Project [c1#x, c2#x] ++- Filter (c1#x = 1) + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +CREATE TABLE integration_test2(c1 INT, c3 STRING) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`integration_test2`, false + + +-- !query +INSERT INTO integration_test2 VALUES (1, 'x'), (2, 'y') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test2, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test2], Append, `spark_catalog`.`identifier_clause_test_schema`.`integration_test2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test2), [c1, c3] ++- Project [col1#x AS c1#x, col2#x AS c3#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +EXECUTE IMMEDIATE 'SELECT t1.*, t2.* FROM IDENTIFIER(:t1) t1 JOIN IDENTIFIER(:t2) t2 USING (IDENTIFIER(:col)) ORDER BY ALL' + USING 'integration_test' AS t1, 'integration_test2' AS t2, 'c1' AS col +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 103, + "fragment" : "SELECT t1.*, t2.* FROM IDENTIFIER(:t1) t1 JOIN IDENTIFIER(:t2) t2 USING (IDENTIFIER(:col)) ORDER BY ALL" + } ] +} + + +-- !query +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:col2), row_number() OVER (PARTITION BY IDENTIFIER(:part) ORDER BY IDENTIFIER(:ord)) as rn FROM integration_test' + USING 'c1' AS col1, 'c2' AS col2, 'c2' AS part, 'c1' AS ord +-- !query analysis +Project [c1#x, c2#x, rn#x] ++- Project [c1#x, c2#x, rn#x, rn#x] + +- Window [row_number() windowspecdefinition(c2#x, c1#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#x], [c2#x], [c1#x ASC NULLS FIRST] + +- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''2''), IDENTIFIER(:agg)(IDENTIFIER(:col)) FROM integration_test GROUP BY IDENTIFIER(:prefix ''2'') ORDER BY ALL' + USING 'c' AS prefix, 'count' AS agg, 'c1' AS col +-- !query analysis +Sort [c2#x ASC NULLS FIRST, count(c1)#xL ASC NULLS FIRST], true ++- Aggregate [c2#x], [c2#x, count(c1#x) AS count(c1)#xL] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM integration_test ORDER BY IDENTIFIER(:col1) DESC, IDENTIFIER(:col2)' + USING 'c1' AS col1, 'c2' AS col2 +-- !query analysis +Sort [c1#x DESC NULLS LAST, c2#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'INSERT INTO integration_test(IDENTIFIER(:col1), IDENTIFIER(:col2)) VALUES (:val1, :val2)' + USING 'c1' AS col1, 'c2' AS col2, 3 AS val1, 'c' AS val2 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing ')'" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "INSERT INTO integration_test(IDENTIFIER(:col1), IDENTIFIER(:col2)) VALUES (:val1, :val2)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(concat(:schema, ''.'', :table, ''.c1'')) FROM VALUES(named_struct(''c1'', 100)) AS IDENTIFIER(:alias)(IDENTIFIER(:schema ''.'' :table))' + USING 'identifier_clause_test_schema' AS schema, 'my_table' AS table, 't' AS alias +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "':'", + "hint" : ": extra input ':'" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 145, + "fragment" : "SELECT IDENTIFIER(concat(:schema, '.', :table, '.c1')) FROM VALUES(named_struct('c1', 100)) AS IDENTIFIER(:alias)(IDENTIFIER(:schema '.' :table))" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'WITH IDENTIFIER(:cte_name)(c1) AS (VALUES(1)) SELECT c1 FROM IDENTIFIER(:cte_name)' + USING 'my_cte' AS cte_name +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "':'", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 82, + "fragment" : "WITH IDENTIFIER(:cte_name)(c1) AS (VALUES(1)) SELECT c1 FROM IDENTIFIER(:cte_name)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'CREATE OR REPLACE TEMPORARY VIEW IDENTIFIER(:view_name)(IDENTIFIER(:col_name)) AS VALUES(1)' + USING 'test_view' AS view_name, 'test_col' AS col_name +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 91, + "fragment" : "CREATE OR REPLACE TEMPORARY VIEW IDENTIFIER(:view_name)(IDENTIFIER(:col_name)) AS VALUES(1)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col) FROM IDENTIFIER(:view)' + USING 'test_col' AS col, 'test_view' AS view +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`test_view`" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 46, + "fragment" : "IDENTIFIER(:view)" + } ] +} + + +-- !query +DROP VIEW test_view +-- !query analysis +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`spark_catalog`.`identifier_clause_test_schema`.`test_view`" + } +} + + +-- !query +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) ADD COLUMN IDENTIFIER(:new_col) INT' + USING 'integration_test' AS tab, 'c4' AS new_col +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 64, + "fragment" : "ALTER TABLE IDENTIFIER(:tab) ADD COLUMN IDENTIFIER(:new_col) INT" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) RENAME COLUMN IDENTIFIER(:old_col) TO IDENTIFIER(:new_col)' + USING 'integration_test' AS tab, 'c4' AS old_col, 'c5' AS new_col +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 87, + "fragment" : "ALTER TABLE IDENTIFIER(:tab) RENAME COLUMN IDENTIFIER(:old_col) TO IDENTIFIER(:new_col)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT map(:key, :val).IDENTIFIER(:key) AS result' + USING 'mykey' AS key, 42 AS val +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "':'", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 49, + "fragment" : "SELECT map(:key, :val).IDENTIFIER(:key) AS result" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:alias ''.c1'') FROM integration_test AS IDENTIFIER(:alias) ORDER BY ALL' + USING 't' AS alias +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "':'", + "hint" : ": extra input ':'" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "SELECT IDENTIFIER(:alias '.c1') FROM integration_test AS IDENTIFIER(:alias) ORDER BY ALL" + } ] +} + + +-- !query +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:p ''2'') FROM IDENTIFIER(:schema ''.'' :tab) WHERE IDENTIFIER(:col1) > 0 ORDER BY IDENTIFIER(:p ''1'')' + USING 'c1' AS col1, 'c' AS p, 'identifier_clause_test_schema' AS schema, 'integration_test' AS tab +-- !query analysis +Sort [c1#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- Filter (c1#x > 0) + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) WHERE IDENTIFIER(concat(:tab_alias, ''.c1'')) > 0 ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table, 'integration_test' AS tab_alias +-- !query analysis +Sort [c1#x ASC NULLS FIRST, c2#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- Filter (c1#x > 0) + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 AS IDENTIFIER(:schema ''.'' :col)' + USING 'identifier_clause_test_schema' AS schema, 'col1' AS col +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 40, + "fragment" : "SELECT 1 AS IDENTIFIER(:schema '.' :col)" + } ] +} + + +-- !query +DROP TABLE integration_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.integration_test + + +-- !query +DROP TABLE integration_test2 +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.integration_test2 + + +-- !query +CREATE TABLE lateral_test(arr ARRAY) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`lateral_test`, false + + +-- !query +INSERT INTO lateral_test VALUES (array(1, 2, 3)) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/lateral_test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/lateral_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`lateral_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/lateral_test), [arr] ++- Project [col1#x AS arr#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT * FROM lateral_test LATERAL VIEW explode(arr) IDENTIFIER('tbl') AS IDENTIFIER('col') ORDER BY ALL +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM lateral_test LATERAL VIEW OUTER explode(arr) IDENTIFIER('my_table') AS IDENTIFIER('my_col') ORDER BY ALL +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE lateral_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.lateral_test + + +-- !query +CREATE TABLE unpivot_test(id INT, a INT, b INT, c INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`unpivot_test`, false + + +-- !query +INSERT INTO unpivot_test VALUES (1, 10, 20, 30) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/unpivot_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/unpivot_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`unpivot_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/unpivot_test), [id, a, b, c] ++- Project [col1#x AS id#x, col2#x AS a#x, col3#x AS b#x, col4#x AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +SELECT * FROM unpivot_test UNPIVOT (val FOR col IN (a AS IDENTIFIER('col_a'), b AS IDENTIFIER('col_b'))) ORDER BY ALL +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM unpivot_test UNPIVOT ((v1, v2) FOR col IN ((a, b) AS IDENTIFIER('cols_ab'), (b, c) AS IDENTIFIER('cols_bc'))) ORDER BY ALL +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE unpivot_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.unpivot_test + + +-- !query +SELECT :IDENTIFIER('param1') FROM VALUES(1) AS T(c1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''param1''", + "hint" : "" + } +} + + +-- !query +CREATE TABLE hint_test(c1 INT, c2 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`hint_test`, false + + +-- !query +INSERT INTO hint_test VALUES (1, 2), (3, 4) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/hint_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/hint_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`hint_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/hint_test), [c1, c2] ++- Project [col1#x AS c1#x, col2#x AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT /*+ IDENTIFIER('BROADCAST')(hint_test) */ * FROM hint_test +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT /*+ IDENTIFIER('MERGE')(hint_test) */ * FROM hint_test +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE hint_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.hint_test + + +-- !query +SHOW IDENTIFIER('USER') FUNCTIONS +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT EXTRACT(IDENTIFIER('YEAR') FROM DATE'2024-01-15') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT TIMESTAMPADD(IDENTIFIER('YEAR'), 1, DATE'2024-01-15') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`TIMESTAMPADD`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`identifier_clause_test_schema`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 60, + "fragment" : "TIMESTAMPADD(IDENTIFIER('YEAR'), 1, DATE'2024-01-15')" + } ] +} + + +-- !query +DROP SCHEMA identifier_clause_test_schema +-- !query analysis +DropNamespace false, false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [identifier_clause_test_schema] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out index 13d911c98838..fc8d225e2271 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out @@ -107,9 +107,11 @@ UPDATE IDENTIFIER('ta' || 'b') SET c1 = 2 -- !query analysis org.apache.spark.SparkUnsupportedOperationException { - "errorClass" : "_LEGACY_ERROR_TEMP_2096", + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", "messageParameters" : { - "ddl" : "UPDATE TABLE" + "operation" : "UPDATE TABLE", + "tableName" : "`spark_catalog`.`s`.`tab`" } } @@ -120,9 +122,11 @@ MERGE INTO IDENTIFIER('ta' || 'b') AS t USING IDENTIFIER('ta' || 'b') AS s ON s. -- !query analysis org.apache.spark.SparkUnsupportedOperationException { - "errorClass" : "_LEGACY_ERROR_TEMP_2096", + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", "messageParameters" : { - "ddl" : "MERGE INTO TABLE" + "operation" : "MERGE INTO TABLE", + "tableName" : "`spark_catalog`.`s`.`tab`" } } @@ -853,7 +857,8 @@ org.apache.spark.sql.AnalysisException "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", "sqlState" : "42601", "messageParameters" : { - "identifier" : "`a`.`b`.`c`.`d`" + "identifier" : "`a`.`b`.`c`.`d`", + "limit" : "2" }, "queryContext" : [ { "objectType" : "", @@ -1064,27 +1069,32 @@ SELECT row_number() OVER IDENTIFIER('x.win') FROM VALUES(1) AS T(c1) WINDOW win -- !query analysis org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "PARSE_SYNTAX_ERROR", + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", "sqlState" : "42601", "messageParameters" : { - "error" : "''x.win''", - "hint" : "" - } + "identifier" : "`x`.`win`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 44, + "fragment" : "IDENTIFIER('x.win')" + } ] } -- !query SELECT T1.c1 FROM VALUES(1) AS T1(c1) JOIN VALUES(1) AS T2(c1) USING (IDENTIFIER('c1')) -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'('", - "hint" : "" - } -} +Project [c1#x] ++- Project [c1#x] + +- Join Inner, (c1#x = c1#x) + :- SubqueryAlias T1 + : +- LocalRelation [c1#x] + +- SubqueryAlias T2 + +- LocalRelation [c1#x] -- !query @@ -1111,40 +1121,28 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT map('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1) -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "''a''", - "hint" : "" - } -} +Project [map(a, 1)[a] AS map(a, 1)[a]#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] -- !query SELECT named_struct('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1) -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "''a''", - "hint" : "" - } -} +Project [named_struct(a, 1).a AS named_struct(a, 1).a#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] -- !query SELECT * FROM s.IDENTIFIER('tab') -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "INVALID_SQL_SYNTAX.INVALID_TABLE_VALUED_FUNC_NAME", - "sqlState" : "42000", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "funcName" : "`s`.`IDENTIFIER`" + "relationName" : "`s`.`tab`" }, "queryContext" : [ { "objectType" : "", @@ -1159,110 +1157,1021 @@ org.apache.spark.sql.catalyst.parser.ParseException -- !query SELECT * FROM IDENTIFIER('s').IDENTIFIER('tab') -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "error" : "'.'", - "hint" : "" - } + "relationName" : "`s`.`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 47, + "fragment" : "IDENTIFIER('s').IDENTIFIER('tab')" + } ] } -- !query SELECT * FROM IDENTIFIER('s').tab -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "error" : "'.'", - "hint" : "" - } + "relationName" : "`s`.`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 33, + "fragment" : "IDENTIFIER('s').tab" + } ] } -- !query SELECT row_number() OVER IDENTIFIER('win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1) -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +Project [row_number() OVER (ORDER BY c1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [c1#x, row_number() OVER (ORDER BY c1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, row_number() OVER (ORDER BY c1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [row_number() windowspecdefinition(c1#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS row_number() OVER (ORDER BY c1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [c1#x ASC NULLS FIRST] + +- Project [c1#x] + +- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +SELECT row_number() OVER win FROM VALUES(1) AS T(c1) WINDOW IDENTIFIER('win') AS (ORDER BY c1) +-- !query analysis +Project [row_number() OVER (ORDER BY c1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [c1#x, row_number() OVER (ORDER BY c1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, row_number() OVER (ORDER BY c1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [row_number() windowspecdefinition(c1#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS row_number() OVER (ORDER BY c1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [c1#x ASC NULLS FIRST] + +- Project [c1#x] + +- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +SELECT 1 AS IDENTIFIER('col1') +-- !query analysis +Project [1 AS col1#x] ++- OneRowRelation + + +-- !query +SELECT my_table.* FROM VALUES (1, 2) AS IDENTIFIER('my_table')(IDENTIFIER('c1'), IDENTIFIER('c2')) +-- !query analysis +Project [c1#x, c2#x] ++- SubqueryAlias my_table + +- LocalRelation [c1#x, c2#x] + + +-- !query +WITH identifier('v')(identifier('c1')) AS (VALUES(1)) (SELECT c1 FROM v) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias v +: +- Project [col1#x AS c1#x] +: +- LocalRelation [col1#x] ++- Project [c1#x] + +- SubqueryAlias v + +- CTERelationRef xxxx, true, [c1#x], false, false, 1 + + +-- !query +CREATE OR REPLACE VIEW v(IDENTIFIER('c1')) AS VALUES(1) +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`v`, [(c1,None)], VALUES(1), false, true, PersistedView, COMPENSATION, true + +- LocalRelation [col1#x] + + +-- !query +SELECT c1 FROM v +-- !query analysis +Project [c1#x] ++- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x]) + +- Project [cast(col1#x as int) AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tab`, false + + +-- !query +INSERT INTO tab(IDENTIFIER('c1')) VALUES(1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tab, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/tab], Append, `spark_catalog`.`default`.`tab`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tab), [c1] ++- Project [c1#x AS c1#x] + +- Project [col1#x AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT c1 FROM tab +-- !query analysis +Project [c1#x] ++- SubqueryAlias spark_catalog.default.tab + +- Relation spark_catalog.default.tab[c1#x] csv + + +-- !query +ALTER TABLE IDENTIFIER('tab') RENAME COLUMN IDENTIFIER('c1') TO IDENTIFIER('col1') +-- !query analysis +org.apache.spark.sql.AnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", "messageParameters" : { - "error" : "''win''", - "hint" : "" + "operation" : "RENAME COLUMN", + "tableName" : "`spark_catalog`.`default`.`tab`" } } -- !query -SELECT row_number() OVER win FROM VALUES(1) AS T(c1) WINDOW IDENTIFIER('win') AS (ORDER BY c1) +SELECT col1 FROM tab -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", "messageParameters" : { - "error" : "'WINDOW'", - "hint" : "" - } + "objectName" : "`col1`", + "proposal" : "`c1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 11, + "fragment" : "col1" + } ] } -- !query -WITH identifier('v')(identifier('c1')) AS (VALUES(1)) (SELECT c1 FROM v) +ALTER TABLE IDENTIFIER('tab') ADD COLUMN IDENTIFIER('c2') INT -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +AlterTableAddColumnsCommand `spark_catalog`.`default`.`tab`, [StructField(c2,IntegerType,true)] + + +-- !query +SELECT c2 FROM tab +-- !query analysis +Project [c2#x] ++- SubqueryAlias spark_catalog.default.tab + +- Relation spark_catalog.default.tab[c1#x,c2#x] csv + + +-- !query +ALTER TABLE IDENTIFIER('tab') DROP COLUMN IDENTIFIER('c2') +-- !query analysis +org.apache.spark.sql.AnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", "messageParameters" : { - "error" : "''v''", - "hint" : "" + "operation" : "DROP COLUMN", + "tableName" : "`spark_catalog`.`default`.`tab`" } } -- !query -INSERT INTO tab(IDENTIFIER('c1')) VALUES(1) +ALTER TABLE IDENTIFIER('tab') RENAME TO IDENTIFIER('tab_renamed') +-- !query analysis +AlterTableRenameCommand `spark_catalog`.`default`.`tab`, `tab_renamed`, false + + +-- !query +SELECT * FROM tab_renamed +-- !query analysis +Project [c1#x, c2#x] ++- SubqueryAlias spark_catalog.default.tab_renamed + +- Relation spark_catalog.default.tab_renamed[c1#x,c2#x] csv + + +-- !query +DROP TABLE IF EXISTS tab_renamed +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tab_renamed + + +-- !query +DROP TABLE IF EXISTS tab +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tab + + +-- !query +CREATE TABLE test_col_with_dot(IDENTIFIER('`col.with.dot`') INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`test_col_with_dot`, false + + +-- !query +DROP TABLE IF EXISTS test_col_with_dot +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.test_col_with_dot + + +-- !query +SELECT * FROM VALUES (1, 2) AS IDENTIFIER('schema.table')(c1, c2) -- !query analysis org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "PARSE_SYNTAX_ERROR", + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", "sqlState" : "42601", "messageParameters" : { - "error" : "'('", - "hint" : ": missing ')'" - } + "identifier" : "`schema`.`table`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 65, + "fragment" : "VALUES (1, 2) AS IDENTIFIER('schema.table')(c1, c2)" + } ] } -- !query -CREATE OR REPLACE VIEW v(IDENTIFIER('c1')) AS VALUES(1) +SELECT 1 AS IDENTIFIER('col1.col2') -- !query analysis org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "PARSE_SYNTAX_ERROR", + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", "sqlState" : "42601", "messageParameters" : { - "error" : "'('", - "hint" : "" - } + "identifier" : "`col1`.`col2`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "1 AS IDENTIFIER('col1.col2')" + } ] } -- !query -CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV +CREATE SCHEMA identifier_clause_test_schema -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [identifier_clause_test_schema] + + +-- !query +USE identifier_clause_test_schema +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [identifier_clause_test_schema] + + +-- !query +CREATE TABLE test_show(c1 INT, c2 STRING) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_show`, false + + +-- !query +SHOW VIEWS IN IDENTIFIER('identifier_clause_test_schema') +-- !query analysis +ShowViewsCommand identifier_clause_test_schema, [namespace#x, viewName#x, isTemporary#x] + + +-- !query +SHOW PARTITIONS IDENTIFIER('test_show') +-- !query analysis +org.apache.spark.sql.AnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", + "errorClass" : "INVALID_PARTITION_OPERATION.PARTITION_SCHEMA_IS_EMPTY", "sqlState" : "42601", "messageParameters" : { - "error" : "'('", - "hint" : "" + "name" : "`spark_catalog`.`identifier_clause_test_schema`.`test_show`" } } + + +-- !query +SHOW CREATE TABLE IDENTIFIER('test_show') +-- !query analysis +ShowCreateTable false, [createtab_stmt#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_show, V1Table(identifier_clause_test_schema.test_show), [c1#x, c2#x] + + +-- !query +DROP TABLE test_show +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_show + + +-- !query +CREATE TABLE test_desc(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, false + + +-- !query +DESCRIBE TABLE IDENTIFIER('test_desc') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESCRIBE FORMATTED IDENTIFIER('test_desc') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESCRIBE EXTENDED IDENTIFIER('test_desc') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC IDENTIFIER('test_desc') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_desc`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DROP TABLE test_desc +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_desc + + +-- !query +CREATE TABLE test_comment(c1 INT, c2 STRING) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_comment`, false + + +-- !query +COMMENT ON TABLE IDENTIFIER('test_comment') IS 'table comment' +-- !query analysis +CommentOnTable table comment ++- ResolvedTable V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_comment, V1Table(identifier_clause_test_schema.test_comment), [c1#x, c2#x] + + +-- !query +ALTER TABLE test_comment ALTER COLUMN IDENTIFIER('c1') COMMENT 'column comment' +-- !query analysis +AlterTableChangeColumnCommand `spark_catalog`.`identifier_clause_test_schema`.`test_comment`, c1, StructField(c1,IntegerType,true) + + +-- !query +DROP TABLE test_comment +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_comment + + +-- !query +CREATE TABLE identifier_clause_test_schema.test_table(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table`, false + + +-- !query +ANALYZE TABLE IDENTIFIER('identifier_clause_test_schema.test_table') COMPUTE STATISTICS +-- !query analysis +AnalyzeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table`, false + + +-- !query +REFRESH TABLE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query analysis +RefreshTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table` + + +-- !query +DESCRIBE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query analysis +DescribeTableCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +SHOW COLUMNS FROM IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query analysis +ShowColumnsCommand `spark_catalog`.`identifier_clause_test_schema`.`test_table`, [col_name#x] + + +-- !query +DROP TABLE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.test_table + + +-- !query +DECLARE IDENTIFIER('my_var') = 'value' +-- !query analysis +CreateVariable defaultvalueexpression(value, 'value'), false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.my_var + + +-- !query +SET VAR IDENTIFIER('my_var') = 'new_value' +-- !query analysis +SetVariable [variablereference(system.session.my_var='value')] ++- Project [new_value AS my_var#x] + +- OneRowRelation + + +-- !query +SELECT IDENTIFIER('my_var') +-- !query analysis +Project [variablereference(system.session.my_var='new_value') AS variablereference(system.session.my_var='new_value')#x] ++- OneRowRelation + + +-- !query +DROP TEMPORARY VARIABLE IDENTIFIER('my_var') +-- !query analysis +DropVariable false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.my_var + + +-- !query +CREATE TEMPORARY FUNCTION test_udf(IDENTIFIER('param1') INT, IDENTIFIER('param2') STRING) +RETURNS INT +RETURN IDENTIFIER('param1') + length(IDENTIFIER('param2')) +-- !query analysis +CreateSQLFunctionCommand test_udf, IDENTIFIER('param1') INT, IDENTIFIER('param2') STRING, INT, IDENTIFIER('param1') + length(IDENTIFIER('param2')), false, true, false, false + + +-- !query +SELECT test_udf(5, 'hello') +-- !query analysis +Project [test_udf(param1#x, param2#x) AS test_udf(5, hello)#x] ++- Project [cast(5 as int) AS param1#x, cast(hello as string) AS param2#x] + +- OneRowRelation + + +-- !query +DROP TEMPORARY FUNCTION test_udf +-- !query analysis +DropFunctionCommand test_udf, false, true + + +-- !query +CREATE TEMPORARY FUNCTION test_table_udf(IDENTIFIER('input_val') INT) +RETURNS TABLE(IDENTIFIER('col1') INT, IDENTIFIER('col2') STRING) +RETURN SELECT IDENTIFIER('input_val'), 'result' +-- !query analysis +CreateSQLFunctionCommand test_table_udf, IDENTIFIER('input_val') INT, IDENTIFIER('col1') INT, IDENTIFIER('col2') STRING, SELECT IDENTIFIER('input_val'), 'result', true, true, false, false + + +-- !query +SELECT * FROM test_table_udf(42) +-- !query analysis +Project [col1#x, col2#x] ++- SQLFunctionNode test_table_udf + +- SubqueryAlias test_table_udf + +- Project [cast(input_val#x as int) AS col1#x, cast(result#x as string) AS col2#x] + +- Project [cast(42 as int) AS input_val#x, result AS result#x] + +- OneRowRelation + + +-- !query +DROP TEMPORARY FUNCTION test_table_udf +-- !query analysis +DropFunctionCommand test_table_udf, false, true + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:tab \'b\').c1 FROM VALUES(1) AS tab(c1)' USING 'ta' AS tab +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_BASE_FIELD_TYPE", + "sqlState" : "42000", + "messageParameters" : { + "base" : "\"variablereference(system.session.tab='T')\"", + "other" : "\"STRING\"" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col1 ''.c2'') FROM VALUES(named_struct(''c2'', 42)) AS T(c1)' + USING 'c1' AS col1 +-- !query analysis +Project [c1#x.c2 AS c1.c2#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +CREATE TABLE integration_test(c1 INT, c2 STRING) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`integration_test`, false + + +-- !query +INSERT INTO integration_test VALUES (1, 'a'), (2, 'b') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`integration_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test), [c1, c2] ++- Project [col1#x AS c1#x, col2#x AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table +-- !query analysis +Sort [c1#x ASC NULLS FIRST, c2#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''1''), IDENTIFIER(:prefix ''2'') FROM integration_test ORDER BY ALL' + USING 'c' AS prefix +-- !query analysis +Sort [c1#x ASC NULLS FIRST, c2#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM integration_test WHERE IDENTIFIER(:col) = :val' + USING 'c1' AS col, 1 AS val +-- !query analysis +Project [c1#x, c2#x] ++- Filter (c1#x = 1) + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +CREATE TABLE integration_test2(c1 INT, c3 STRING) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`integration_test2`, false + + +-- !query +INSERT INTO integration_test2 VALUES (1, 'x'), (2, 'y') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test2, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test2], Append, `spark_catalog`.`identifier_clause_test_schema`.`integration_test2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test2), [c1, c3] ++- Project [col1#x AS c1#x, col2#x AS c3#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +EXECUTE IMMEDIATE 'SELECT t1.*, t2.* FROM IDENTIFIER(:t1) t1 JOIN IDENTIFIER(:t2) t2 USING (IDENTIFIER(:col)) ORDER BY ALL' + USING 'integration_test' AS t1, 'integration_test2' AS t2, 'c1' AS col +-- !query analysis +Sort [c1#x ASC NULLS FIRST, c2#x ASC NULLS FIRST, c1#x ASC NULLS FIRST, c3#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x, c1#x, c3#x] + +- Project [c1#x, c2#x, c3#x, c1#x] + +- Join Inner, (c1#x = c1#x) + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + : +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test2 + +- Relation spark_catalog.identifier_clause_test_schema.integration_test2[c1#x,c3#x] csv + + +-- !query +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:col2), row_number() OVER (PARTITION BY IDENTIFIER(:part) ORDER BY IDENTIFIER(:ord)) as rn FROM integration_test' + USING 'c1' AS col1, 'c2' AS col2, 'c2' AS part, 'c1' AS ord +-- !query analysis +Project [c1#x, c2#x, rn#x] ++- Project [c1#x, c2#x, rn#x, rn#x] + +- Window [row_number() windowspecdefinition(c2#x, c1#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#x], [c2#x], [c1#x ASC NULLS FIRST] + +- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''2''), IDENTIFIER(:agg)(IDENTIFIER(:col)) FROM integration_test GROUP BY IDENTIFIER(:prefix ''2'') ORDER BY ALL' + USING 'c' AS prefix, 'count' AS agg, 'c1' AS col +-- !query analysis +Sort [c2#x ASC NULLS FIRST, count(c1)#xL ASC NULLS FIRST], true ++- Aggregate [c2#x], [c2#x, count(c1#x) AS count(c1)#xL] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM integration_test ORDER BY IDENTIFIER(:col1) DESC, IDENTIFIER(:col2)' + USING 'c1' AS col1, 'c2' AS col2 +-- !query analysis +Sort [c1#x DESC NULLS LAST, c2#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x] csv + + +-- !query +EXECUTE IMMEDIATE 'INSERT INTO integration_test(IDENTIFIER(:col1), IDENTIFIER(:col2)) VALUES (:val1, :val2)' + USING 'c1' AS col1, 'c2' AS col2, 3 AS val1, 'c' AS val2 +-- !query analysis +CommandResult Execute InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`integration_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test), [c1, c2] + +- InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`integration_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/integration_test), [c1, c2] + +- Project [c1#x AS c1#x, c2#x AS c2#x] + +- Project [col1#x AS c1#x, col2#x AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(concat(:schema, ''.'', :table, ''.c1'')) FROM VALUES(named_struct(''c1'', 100)) AS IDENTIFIER(:alias)(IDENTIFIER(:schema ''.'' :table))' + USING 'identifier_clause_test_schema' AS schema, 'my_table' AS table, 't' AS alias +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", + "sqlState" : "42601", + "messageParameters" : { + "identifier" : "`identifier_clause_test_schema`.`my_table`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 115, + "stopIndex" : 144, + "fragment" : "IDENTIFIER(:schema '.' :table)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'WITH IDENTIFIER(:cte_name)(c1) AS (VALUES(1)) SELECT c1 FROM IDENTIFIER(:cte_name)' + USING 'my_cte' AS cte_name +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias my_cte +: +- Project [col1#x AS c1#x] +: +- LocalRelation [col1#x] ++- Project [c1#x] + +- SubqueryAlias my_cte + +- CTERelationRef xxxx, true, [c1#x], false, false, 1 + + +-- !query +EXECUTE IMMEDIATE 'CREATE OR REPLACE TEMPORARY VIEW IDENTIFIER(:view_name)(IDENTIFIER(:col_name)) AS VALUES(1)' + USING 'test_view' AS view_name, 'test_col' AS col_name +-- !query analysis +CommandResult Execute CreateViewCommand + +- CreateViewCommand `test_view`, [(test_col,None)], VALUES(1), false, true, LocalTempView, UNSUPPORTED, true + +- LocalRelation [col1#x] + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col) FROM IDENTIFIER(:view)' + USING 'test_col' AS col, 'test_view' AS view +-- !query analysis +Project [test_col#x] ++- SubqueryAlias test_view + +- View (`test_view`, [test_col#x]) + +- Project [cast(col1#x as int) AS test_col#x] + +- LocalRelation [col1#x] + + +-- !query +DROP VIEW test_view +-- !query analysis +DropTempViewCommand test_view + + +-- !query +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) ADD COLUMN IDENTIFIER(:new_col) INT' + USING 'integration_test' AS tab, 'c4' AS new_col +-- !query analysis +CommandResult Execute AlterTableAddColumnsCommand + +- AlterTableAddColumnsCommand `spark_catalog`.`identifier_clause_test_schema`.`integration_test`, [StructField(c4,IntegerType,true)] + + +-- !query +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) RENAME COLUMN IDENTIFIER(:old_col) TO IDENTIFIER(:new_col)' + USING 'integration_test' AS tab, 'c4' AS old_col, 'c5' AS new_col +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "RENAME COLUMN", + "tableName" : "`spark_catalog`.`identifier_clause_test_schema`.`integration_test`" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT map(:key, :val).IDENTIFIER(:key) AS result' + USING 'mykey' AS key, 42 AS val +-- !query analysis +Project [map(mykey, 42)[mykey] AS result#x] ++- OneRowRelation + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:alias ''.c1'') FROM integration_test AS IDENTIFIER(:alias) ORDER BY ALL' + USING 't' AS alias +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t`.`c1`", + "proposal" : "`IDENTIFIER('t')`.`c1`, `IDENTIFIER('t')`.`c2`, `IDENTIFIER('t')`.`c4`" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "IDENTIFIER(:alias '.c1')" + } ] +} + + +-- !query +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:p ''2'') FROM IDENTIFIER(:schema ''.'' :tab) WHERE IDENTIFIER(:col1) > 0 ORDER BY IDENTIFIER(:p ''1'')' + USING 'c1' AS col1, 'c' AS p, 'identifier_clause_test_schema' AS schema, 'integration_test' AS tab +-- !query analysis +Sort [c1#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- Filter (c1#x > 0) + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x,c4#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) WHERE IDENTIFIER(concat(:tab_alias, ''.c1'')) > 0 ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table, 'integration_test' AS tab_alias +-- !query analysis +Sort [c1#x ASC NULLS FIRST, c2#x ASC NULLS FIRST, c4#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x, c4#x] + +- Filter (c1#x > 0) + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.integration_test + +- Relation spark_catalog.identifier_clause_test_schema.integration_test[c1#x,c2#x,c4#x] csv + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 AS IDENTIFIER(:schema ''.'' :col)' + USING 'identifier_clause_test_schema' AS schema, 'col1' AS col +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", + "sqlState" : "42601", + "messageParameters" : { + "identifier" : "`identifier_clause_test_schema`.`col1`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "1 AS IDENTIFIER(:schema '.' :col)" + } ] +} + + +-- !query +DROP TABLE integration_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.integration_test + + +-- !query +DROP TABLE integration_test2 +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.integration_test2 + + +-- !query +CREATE TABLE lateral_test(arr ARRAY) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`lateral_test`, false + + +-- !query +INSERT INTO lateral_test VALUES (array(1, 2, 3)) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/lateral_test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/lateral_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`lateral_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/lateral_test), [arr] ++- Project [col1#x AS arr#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT * FROM lateral_test LATERAL VIEW explode(arr) IDENTIFIER('tbl') AS IDENTIFIER('col') ORDER BY ALL +-- !query analysis +Sort [arr#x ASC NULLS FIRST, col#x ASC NULLS FIRST], true ++- Project [arr#x, col#x] + +- Generate explode(arr#x), false, tbl, [col#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.lateral_test + +- Relation spark_catalog.identifier_clause_test_schema.lateral_test[arr#x] parquet + + +-- !query +SELECT * FROM lateral_test LATERAL VIEW OUTER explode(arr) IDENTIFIER('my_table') AS IDENTIFIER('my_col') ORDER BY ALL +-- !query analysis +Sort [arr#x ASC NULLS FIRST, my_col#x ASC NULLS FIRST], true ++- Project [arr#x, my_col#x] + +- Generate explode(arr#x), true, my_table, [my_col#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.lateral_test + +- Relation spark_catalog.identifier_clause_test_schema.lateral_test[arr#x] parquet + + +-- !query +DROP TABLE lateral_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.lateral_test + + +-- !query +CREATE TABLE unpivot_test(id INT, a INT, b INT, c INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`unpivot_test`, false + + +-- !query +INSERT INTO unpivot_test VALUES (1, 10, 20, 30) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/unpivot_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/unpivot_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`unpivot_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/unpivot_test), [id, a, b, c] ++- Project [col1#x AS id#x, col2#x AS a#x, col3#x AS b#x, col4#x AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +SELECT * FROM unpivot_test UNPIVOT (val FOR col IN (a AS IDENTIFIER('col_a'), b AS IDENTIFIER('col_b'))) ORDER BY ALL +-- !query analysis +Sort [id#x ASC NULLS FIRST, c#x ASC NULLS FIRST, col#x ASC NULLS FIRST, val#x ASC NULLS FIRST], true ++- Project [id#x, c#x, col#x, val#x] + +- Filter isnotnull(coalesce(val#x)) + +- Expand [[id#x, c#x, col_a, a#x], [id#x, c#x, col_b, b#x]], [id#x, c#x, col#x, val#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.unpivot_test + +- Relation spark_catalog.identifier_clause_test_schema.unpivot_test[id#x,a#x,b#x,c#x] csv + + +-- !query +SELECT * FROM unpivot_test UNPIVOT ((v1, v2) FOR col IN ((a, b) AS IDENTIFIER('cols_ab'), (b, c) AS IDENTIFIER('cols_bc'))) ORDER BY ALL +-- !query analysis +Sort [id#x ASC NULLS FIRST, col#x ASC NULLS FIRST, v1#x ASC NULLS FIRST, v2#x ASC NULLS FIRST], true ++- Project [id#x, col#x, v1#x, v2#x] + +- Filter isnotnull(coalesce(v1#x, v2#x)) + +- Expand [[id#x, cols_ab, a#x, b#x], [id#x, cols_bc, b#x, c#x]], [id#x, col#x, v1#x, v2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.unpivot_test + +- Relation spark_catalog.identifier_clause_test_schema.unpivot_test[id#x,a#x,b#x,c#x] csv + + +-- !query +DROP TABLE unpivot_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.unpivot_test + + +-- !query +SELECT :IDENTIFIER('param1') FROM VALUES(1) AS T(c1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''param1''", + "hint" : "" + } +} + + +-- !query +CREATE TABLE hint_test(c1 INT, c2 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`hint_test`, false + + +-- !query +INSERT INTO hint_test VALUES (1, 2), (3, 4) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/hint_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/hint_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`hint_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/hint_test), [c1, c2] ++- Project [col1#x AS c1#x, col2#x AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT /*+ IDENTIFIER('BROADCAST')(hint_test) */ * FROM hint_test +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT /*+ IDENTIFIER('MERGE')(hint_test) */ * FROM hint_test +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE hint_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.hint_test + + +-- !query +SHOW IDENTIFIER('USER') FUNCTIONS +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT EXTRACT(IDENTIFIER('YEAR') FROM DATE'2024-01-15') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'FROM'", + "hint" : ": missing ')'" + } +} + + +-- !query +SELECT TIMESTAMPADD(IDENTIFIER('YEAR'), 1, DATE'2024-01-15') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`TIMESTAMPADD`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`identifier_clause_test_schema`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 60, + "fragment" : "TIMESTAMPADD(IDENTIFIER('YEAR'), 1, DATE'2024-01-15')" + } ] +} + + +-- !query +DROP SCHEMA identifier_clause_test_schema +-- !query analysis +DropNamespace false, false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [identifier_clause_test_schema] diff --git a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql index 17fa47be4eec..16e1850d5e59 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql @@ -289,3 +289,11 @@ EXECUTE IMMEDIATE 'SELECT typeof(:p) as type, :p as val' USING MAP('key1', 'valu -- !query EXECUTE IMMEDIATE 'SELECT typeof(:p) as type, :p as val' USING MAP(1, 'one', 2, 'two') AS p; +-- !query +-- Test unbound parameter markers without USING clause +-- named parameter without USING clause should fail +EXECUTE IMMEDIATE 'SELECT :param'; + +-- !query +-- positional parameter without USING clause should fail +EXECUTE IMMEDIATE 'SELECT ?'; diff --git a/sql/core/src/test/resources/sql-tests/inputs/identifier-clause-legacy.sql b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause-legacy.sql new file mode 100644 index 000000000000..ae1f10f1af1f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause-legacy.sql @@ -0,0 +1,2 @@ +--SET spark.sql.legacy.identifierClause = true +--IMPORT identifier-clause.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql index 4aa8019097fd..c90165d31e90 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql @@ -157,7 +157,6 @@ SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('T'); WITH ABC(c1, c2) AS (VALUES(1, 2), (2, 3)) SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('A' || 'BC'); --- Not supported SELECT row_number() OVER IDENTIFIER('x.win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1); SELECT T1.c1 FROM VALUES(1) AS T1(c1) JOIN VALUES(1) AS T2(c1) USING (IDENTIFIER('c1')); SELECT IDENTIFIER('t').c1 FROM VALUES(1) AS T(c1); @@ -168,10 +167,216 @@ SELECT * FROM IDENTIFIER('s').IDENTIFIER('tab'); SELECT * FROM IDENTIFIER('s').tab; SELECT row_number() OVER IDENTIFIER('win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1); SELECT row_number() OVER win FROM VALUES(1) AS T(c1) WINDOW IDENTIFIER('win') AS (ORDER BY c1); +SELECT 1 AS IDENTIFIER('col1'); +SELECT my_table.* FROM VALUES (1, 2) AS IDENTIFIER('my_table')(IDENTIFIER('c1'), IDENTIFIER('c2')); WITH identifier('v')(identifier('c1')) AS (VALUES(1)) (SELECT c1 FROM v); -INSERT INTO tab(IDENTIFIER('c1')) VALUES(1); CREATE OR REPLACE VIEW v(IDENTIFIER('c1')) AS VALUES(1); +SELECT c1 FROM v; CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV; +INSERT INTO tab(IDENTIFIER('c1')) VALUES(1); +SELECT c1 FROM tab; +ALTER TABLE IDENTIFIER('tab') RENAME COLUMN IDENTIFIER('c1') TO IDENTIFIER('col1'); +SELECT col1 FROM tab; +ALTER TABLE IDENTIFIER('tab') ADD COLUMN IDENTIFIER('c2') INT; +SELECT c2 FROM tab; +ALTER TABLE IDENTIFIER('tab') DROP COLUMN IDENTIFIER('c2'); +ALTER TABLE IDENTIFIER('tab') RENAME TO IDENTIFIER('tab_renamed'); +SELECT * FROM tab_renamed; +DROP TABLE IF EXISTS tab_renamed; +DROP TABLE IF EXISTS tab; + +-- Error because qualified names are not allowed +CREATE TABLE test_col_with_dot(IDENTIFIER('`col.with.dot`') INT) USING CSV; +DROP TABLE IF EXISTS test_col_with_dot; +-- Identifier-lite: table alias with qualified name should error (table alias must be single) +SELECT * FROM VALUES (1, 2) AS IDENTIFIER('schema.table')(c1, c2); +-- Identifier-lite: column alias with qualified name should error (column alias must be single) +SELECT 1 AS IDENTIFIER('col1.col2'); + +-- Additional coverage: SHOW commands with identifier-lite +CREATE SCHEMA identifier_clause_test_schema; +USE identifier_clause_test_schema; +CREATE TABLE test_show(c1 INT, c2 STRING) USING CSV; +SHOW VIEWS IN IDENTIFIER('identifier_clause_test_schema'); +SHOW PARTITIONS IDENTIFIER('test_show'); +SHOW CREATE TABLE IDENTIFIER('test_show'); +DROP TABLE test_show; + +-- SET CATALOG with identifier-lite +-- SET CATALOG IDENTIFIER('spark_catalog'); + +-- DESCRIBE with different forms +CREATE TABLE test_desc(c1 INT) USING CSV; +DESCRIBE TABLE IDENTIFIER('test_desc'); +DESCRIBE FORMATTED IDENTIFIER('test_desc'); +DESCRIBE EXTENDED IDENTIFIER('test_desc'); +DESC IDENTIFIER('test_desc'); +DROP TABLE test_desc; + +-- COMMENT ON COLUMN with identifier-lite +CREATE TABLE test_comment(c1 INT, c2 STRING) USING CSV; +COMMENT ON TABLE IDENTIFIER('test_comment') IS 'table comment'; +ALTER TABLE test_comment ALTER COLUMN IDENTIFIER('c1') COMMENT 'column comment'; +DROP TABLE test_comment; + +-- Additional identifier tests with qualified table names in various commands +CREATE TABLE identifier_clause_test_schema.test_table(c1 INT) USING CSV; +ANALYZE TABLE IDENTIFIER('identifier_clause_test_schema.test_table') COMPUTE STATISTICS; +REFRESH TABLE IDENTIFIER('identifier_clause_test_schema.test_table'); +DESCRIBE IDENTIFIER('identifier_clause_test_schema.test_table'); +SHOW COLUMNS FROM IDENTIFIER('identifier_clause_test_schema.test_table'); +DROP TABLE IDENTIFIER('identifier_clause_test_schema.test_table'); + +-- Session variables with identifier-lite +DECLARE IDENTIFIER('my_var') = 'value'; +SET VAR IDENTIFIER('my_var') = 'new_value'; +SELECT IDENTIFIER('my_var'); +DROP TEMPORARY VARIABLE IDENTIFIER('my_var'); + +-- SQL UDF with identifier-lite in parameter names and return statement +CREATE TEMPORARY FUNCTION test_udf(IDENTIFIER('param1') INT, IDENTIFIER('param2') STRING) +RETURNS INT +RETURN IDENTIFIER('param1') + length(IDENTIFIER('param2')); + +SELECT test_udf(5, 'hello'); +DROP TEMPORARY FUNCTION test_udf; + +-- SQL UDF with table return type using identifier-lite +CREATE TEMPORARY FUNCTION test_table_udf(IDENTIFIER('input_val') INT) +RETURNS TABLE(IDENTIFIER('col1') INT, IDENTIFIER('col2') STRING) +RETURN SELECT IDENTIFIER('input_val'), 'result'; + +SELECT * FROM test_table_udf(42); +DROP TEMPORARY FUNCTION test_table_udf; + +-- Integration tests: Combining parameter markers, string coalescing, and IDENTIFIER +-- These tests demonstrate the power of combining IDENTIFIER with parameters + +-- Test 1: IDENTIFIER with parameter marker for table name +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:tab \'b\').c1 FROM VALUES(1) AS tab(c1)' USING 'ta' AS tab; + +-- Test 2: IDENTIFIER with string coalescing for column name +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col1 ''.c2'') FROM VALUES(named_struct(''c2'', 42)) AS T(c1)' + USING 'c1' AS col1; + +-- Test 3: IDENTIFIER with parameter and string literal coalescing for qualified table name +CREATE TABLE integration_test(c1 INT, c2 STRING) USING CSV; +INSERT INTO integration_test VALUES (1, 'a'), (2, 'b'); +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table; + +-- Test 4: IDENTIFIER in column reference with parameter and string coalescing +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''1''), IDENTIFIER(:prefix ''2'') FROM integration_test ORDER BY ALL' + USING 'c' AS prefix; + +-- Test 5: IDENTIFIER in WHERE clause with parameters +EXECUTE IMMEDIATE 'SELECT * FROM integration_test WHERE IDENTIFIER(:col) = :val' + USING 'c1' AS col, 1 AS val; + +-- Test 6: IDENTIFIER in JOIN with parameters for table and column names +CREATE TABLE integration_test2(c1 INT, c3 STRING) USING CSV; +INSERT INTO integration_test2 VALUES (1, 'x'), (2, 'y'); +EXECUTE IMMEDIATE 'SELECT t1.*, t2.* FROM IDENTIFIER(:t1) t1 JOIN IDENTIFIER(:t2) t2 USING (IDENTIFIER(:col)) ORDER BY ALL' + USING 'integration_test' AS t1, 'integration_test2' AS t2, 'c1' AS col; + +-- Test 7: IDENTIFIER in window function with parameter for partition column +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:col2), row_number() OVER (PARTITION BY IDENTIFIER(:part) ORDER BY IDENTIFIER(:ord)) as rn FROM integration_test' + USING 'c1' AS col1, 'c2' AS col2, 'c2' AS part, 'c1' AS ord; + +-- Test 8: IDENTIFIER in aggregate function with string coalescing +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''2''), IDENTIFIER(:agg)(IDENTIFIER(:col)) FROM integration_test GROUP BY IDENTIFIER(:prefix ''2'') ORDER BY ALL' + USING 'c' AS prefix, 'count' AS agg, 'c1' AS col; + +-- Test 9: IDENTIFIER in ORDER BY with multiple parameters +EXECUTE IMMEDIATE 'SELECT * FROM integration_test ORDER BY IDENTIFIER(:col1) DESC, IDENTIFIER(:col2)' + USING 'c1' AS col1, 'c2' AS col2; + +-- Test 10: IDENTIFIER in INSERT with parameter for column name +EXECUTE IMMEDIATE 'INSERT INTO integration_test(IDENTIFIER(:col1), IDENTIFIER(:col2)) VALUES (:val1, :val2)' + USING 'c1' AS col1, 'c2' AS col2, 3 AS val1, 'c' AS val2; + +-- Test 11: Complex - IDENTIFIER with nested string operations +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(concat(:schema, ''.'', :table, ''.c1'')) FROM VALUES(named_struct(''c1'', 100)) AS IDENTIFIER(:alias)(IDENTIFIER(:schema ''.'' :table))' + USING 'identifier_clause_test_schema' AS schema, 'my_table' AS table, 't' AS alias; + +-- Test 12: IDENTIFIER in CTE name with parameter +EXECUTE IMMEDIATE 'WITH IDENTIFIER(:cte_name)(c1) AS (VALUES(1)) SELECT c1 FROM IDENTIFIER(:cte_name)' + USING 'my_cte' AS cte_name; + +-- Test 13: IDENTIFIER in view name with parameter +EXECUTE IMMEDIATE 'CREATE OR REPLACE TEMPORARY VIEW IDENTIFIER(:view_name)(IDENTIFIER(:col_name)) AS VALUES(1)' + USING 'test_view' AS view_name, 'test_col' AS col_name; +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col) FROM IDENTIFIER(:view)' + USING 'test_col' AS col, 'test_view' AS view; +DROP VIEW test_view; + +-- Test 14: IDENTIFIER in ALTER TABLE with parameters +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) ADD COLUMN IDENTIFIER(:new_col) INT' + USING 'integration_test' AS tab, 'c4' AS new_col; +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) RENAME COLUMN IDENTIFIER(:old_col) TO IDENTIFIER(:new_col)' + USING 'integration_test' AS tab, 'c4' AS old_col, 'c5' AS new_col; + +-- Test 15: IDENTIFIER with dereference using parameters +EXECUTE IMMEDIATE 'SELECT map(:key, :val).IDENTIFIER(:key) AS result' + USING 'mykey' AS key, 42 AS val; + +-- Test 16: IDENTIFIER in table alias with string coalescing +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:alias ''.c1'') FROM integration_test AS IDENTIFIER(:alias) ORDER BY ALL' + USING 't' AS alias; + +-- Test 17: Multiple IDENTIFIER clauses with different parameter combinations +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:p ''2'') FROM IDENTIFIER(:schema ''.'' :tab) WHERE IDENTIFIER(:col1) > 0 ORDER BY IDENTIFIER(:p ''1'')' + USING 'c1' AS col1, 'c' AS p, 'identifier_clause_test_schema' AS schema, 'integration_test' AS tab; + +-- Test 19: IDENTIFIER with qualified name coalescing for schema.table.column pattern +-- This should work for multi-part identifiers +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) WHERE IDENTIFIER(concat(:tab_alias, ''.c1'')) > 0 ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table, 'integration_test' AS tab_alias; + +-- Test 20: Error case - IDENTIFIER with too many parts from parameter coalescing +-- This should error as column alias must be single identifier +EXECUTE IMMEDIATE 'SELECT 1 AS IDENTIFIER(:schema ''.'' :col)' + USING 'identifier_clause_test_schema' AS schema, 'col1' AS col; + +-- Cleanup +DROP TABLE integration_test; +DROP TABLE integration_test2; + +-- LATERAL VIEW with IDENTIFIER() for table and column names +CREATE TABLE lateral_test(arr ARRAY) USING PARQUET; +INSERT INTO lateral_test VALUES (array(1, 2, 3)); +SELECT * FROM lateral_test LATERAL VIEW explode(arr) IDENTIFIER('tbl') AS IDENTIFIER('col') ORDER BY ALL; +SELECT * FROM lateral_test LATERAL VIEW OUTER explode(arr) IDENTIFIER('my_table') AS IDENTIFIER('my_col') ORDER BY ALL; +DROP TABLE lateral_test; + +-- UNPIVOT with IDENTIFIER() for value column alias +CREATE TABLE unpivot_test(id INT, a INT, b INT, c INT) USING CSV; +INSERT INTO unpivot_test VALUES (1, 10, 20, 30); +SELECT * FROM unpivot_test UNPIVOT (val FOR col IN (a AS IDENTIFIER('col_a'), b AS IDENTIFIER('col_b'))) ORDER BY ALL; +SELECT * FROM unpivot_test UNPIVOT ((v1, v2) FOR col IN ((a, b) AS IDENTIFIER('cols_ab'), (b, c) AS IDENTIFIER('cols_bc'))) ORDER BY ALL; +DROP TABLE unpivot_test; + +-- All the following tests fail because they are not about "true" identifiers + +-- This should fail - named parameters don't support IDENTIFIER() +SELECT :IDENTIFIER('param1') FROM VALUES(1) AS T(c1); + +-- Hint names use simpleIdentifier - these should fail +CREATE TABLE hint_test(c1 INT, c2 INT) USING CSV; +INSERT INTO hint_test VALUES (1, 2), (3, 4); +SELECT /*+ IDENTIFIER('BROADCAST')(hint_test) */ * FROM hint_test; +SELECT /*+ IDENTIFIER('MERGE')(hint_test) */ * FROM hint_test; +DROP TABLE hint_test; + +-- These should fail - function scope doesn't support IDENTIFIER() +SHOW IDENTIFIER('USER') FUNCTIONS; +-- EXTRACT field name uses simpleIdentifier - should fail +SELECT EXTRACT(IDENTIFIER('YEAR') FROM DATE'2024-01-15'); +-- TIMESTAMPADD unit is a token, not identifier - should fail +SELECT TIMESTAMPADD(IDENTIFIER('YEAR'), 1, DATE'2024-01-15'); +DROP SCHEMA identifier_clause_test_schema; diff --git a/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out index 06adf4435046..dd1207b4f2be 100644 --- a/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out @@ -1211,3 +1211,47 @@ EXECUTE IMMEDIATE 'SELECT typeof(:p) as type, :p as val' USING MAP(1, 'one', 2, struct> -- !query output map {1:"one",2:"two"} + + +-- !query +EXECUTE IMMEDIATE 'SELECT :param' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNBOUND_SQL_PARAMETER", + "sqlState" : "42P02", + "messageParameters" : { + "name" : "param" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 13, + "fragment" : ":param" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT ?' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNBOUND_SQL_PARAMETER", + "sqlState" : "42P02", + "messageParameters" : { + "name" : "_7" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 8, + "fragment" : "?" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out new file mode 100644 index 000000000000..ad351074692f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/identifier-clause-legacy.sql.out @@ -0,0 +1,2736 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SET hivevar:colname = 'c' +-- !query schema +struct +-- !query output +hivevar:colname 'c' + + +-- !query +SELECT IDENTIFIER(${colname} || '_1') FROM VALUES(1) AS T(c_1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('c1') FROM VALUES(1) AS T(c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('t.c1') FROM VALUES(1) AS T(c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('`t`.c1') FROM VALUES(1) AS T(c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('`c 1`') FROM VALUES(1) AS T(`c 1`) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('``') FROM VALUES(1) AS T(``) +-- !query schema +struct<:int> +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('c' || '1') FROM VALUES(1) AS T(c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +CREATE SCHEMA IF NOT EXISTS s +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE s.tab(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +USE SCHEMA s +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO IDENTIFIER('ta' || 'b') VALUES(1) +-- !query schema +struct<> +-- !query output + + + +-- !query +DELETE FROM IDENTIFIER('ta' || 'b') WHERE 1=0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "DELETE", + "tableName" : "`spark_catalog`.`s`.`tab`" + } +} + + +-- !query +UPDATE IDENTIFIER('ta' || 'b') SET c1 = 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUnsupportedOperationException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "UPDATE TABLE", + "tableName" : "`spark_catalog`.`s`.`tab`" + } +} + + +-- !query +MERGE INTO IDENTIFIER('ta' || 'b') AS t USING IDENTIFIER('ta' || 'b') AS s ON s.c1 = t.c1 + WHEN MATCHED THEN UPDATE SET c1 = 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUnsupportedOperationException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "MERGE INTO TABLE", + "tableName" : "`spark_catalog`.`s`.`tab`" + } +} + + +-- !query +SELECT * FROM IDENTIFIER('tab') +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM IDENTIFIER('s.tab') +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM IDENTIFIER('`s`.`tab`') +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM IDENTIFIER('t' || 'a' || 'b') +-- !query schema +struct +-- !query output +1 + + +-- !query +USE SCHEMA default +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE s.tab +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP SCHEMA s +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT IDENTIFIER('COAL' || 'ESCE')(NULL, 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('abs')(c1) FROM VALUES(-1) AS T(c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM IDENTIFIER('ra' || 'nge')(0, 1) +-- !query schema +struct +-- !query output +0 + + +-- !query +CREATE TABLE IDENTIFIER('tab')(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE SCHEMA identifier_clauses +-- !query schema +struct<> +-- !query output + + + +-- !query +USE identifier_clauses +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE IDENTIFIER('ta' || 'b')(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS IDENTIFIER('identifier_clauses.' || 'tab') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +REPLACE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "REPLACE TABLE", + "tableName" : "`spark_catalog`.`identifier_clauses`.`tab`" + } +} + + +-- !query +CACHE TABLE IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +UNCACHE TABLE IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +USE default +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP SCHEMA identifier_clauses +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tab(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO tab VALUES (1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT c1 FROM tab +-- !query schema +struct +-- !query output +1 + + +-- !query +DESCRIBE IDENTIFIER('ta' || 'b') +-- !query schema +struct +-- !query output +c1 int + + +-- !query +ANALYZE TABLE IDENTIFIER('ta' || 'b') COMPUTE STATISTICS +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER TABLE IDENTIFIER('ta' || 'b') ADD COLUMN c2 INT +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW TBLPROPERTIES IDENTIFIER('ta' || 'b') +-- !query schema +struct +-- !query output + + + +-- !query +SHOW COLUMNS FROM IDENTIFIER('ta' || 'b') +-- !query schema +struct +-- !query output +c1 +c2 + + +-- !query +COMMENT ON TABLE IDENTIFIER('ta' || 'b') IS 'hello' +-- !query schema +struct<> +-- !query output + + + +-- !query +REFRESH TABLE IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +REPAIR TABLE IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_PARTITIONED_TABLE", + "sqlState" : "42809", + "messageParameters" : { + "operation" : "MSCK REPAIR TABLE", + "tableIdentWithDB" : "`spark_catalog`.`default`.`tab`" + } +} + + +-- !query +TRUNCATE TABLE IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS tab +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW IDENTIFIER('v')(c1) AS VALUES(1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 + + +-- !query +ALTER VIEW IDENTIFIER('v') AS VALUES(2) +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IDENTIFIER('v') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW IDENTIFIER('v')(c1) AS VALUES(1) +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IDENTIFIER('v') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE SCHEMA IDENTIFIER('id' || 'ent') +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER SCHEMA IDENTIFIER('id' || 'ent') SET PROPERTIES (somekey = 'somevalue') +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER SCHEMA IDENTIFIER('id' || 'ent') SET LOCATION 'someloc' +-- !query schema +struct<> +-- !query output + + + +-- !query +COMMENT ON SCHEMA IDENTIFIER('id' || 'ent') IS 'some comment' +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE SCHEMA IDENTIFIER('id' || 'ent') +-- !query schema +struct +-- !query output +Catalog Name spark_catalog +Comment some comment +Location [not included in comparison]/{warehouse_dir}/someloc +Namespace Name ident +Owner [not included in comparison] + + +-- !query +SHOW TABLES IN IDENTIFIER('id' || 'ent') +-- !query schema +struct +-- !query output + + + +-- !query +SHOW TABLE EXTENDED IN IDENTIFIER('id' || 'ent') LIKE 'hello' +-- !query schema +struct +-- !query output + + + +-- !query +USE IDENTIFIER('id' || 'ent') +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW CURRENT SCHEMA +-- !query schema +struct +-- !query output +spark_catalog ident + + +-- !query +USE SCHEMA IDENTIFIER('id' || 'ent') +-- !query schema +struct<> +-- !query output + + + +-- !query +USE SCHEMA default +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP SCHEMA IDENTIFIER('id' || 'ent') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE SCHEMA ident +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') +-- !query schema +struct +-- !query output +Class: test.org.apache.spark.sql.MyDoubleAvg +Function: spark_catalog.ident.mydoubleavg +Usage: N/A. + + +-- !query +REFRESH FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP SCHEMA ident +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY FUNCTION IDENTIFIER('my' || 'DoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY FUNCTION IDENTIFIER('my' || 'DoubleAvg') +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE var = 'sometable' +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE IDENTIFIER(var)(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VAR var = 'c1' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT IDENTIFIER(var) FROM VALUES(1) AS T(c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SET VAR var = 'some' +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IDENTIFIER(var || 'table') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT IDENTIFIER('c 1') FROM VALUES(1) AS T(`c 1`) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'1'", + "hint" : ": extra input '1'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "IDENTIFIER('c 1')" + } ] +} + + +-- !query +SELECT IDENTIFIER('') FROM VALUES(1) AS T(``) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_EMPTY_STATEMENT", + "sqlState" : "42617", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "IDENTIFIER('')" + } ] +} + + +-- !query +VALUES(IDENTIFIER(CAST(NULL AS STRING))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NULL", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "CAST(NULL AS STRING)", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 38, + "fragment" : "CAST(NULL AS STRING)" + } ] +} + + +-- !query +VALUES(IDENTIFIER(1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.WRONG_TYPE", + "sqlState" : "42601", + "messageParameters" : { + "dataType" : "int", + "expr" : "1", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 19, + "fragment" : "1" + } ] +} + + +-- !query +VALUES(IDENTIFIER(SUBSTR('HELLO', 1, RAND() + 1))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "substr('HELLO', 1, CAST((rand() + CAST(1 AS DOUBLE)) AS INT))", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 48, + "fragment" : "SUBSTR('HELLO', 1, RAND() + 1)" + } ] +} + + +-- !query +SELECT `IDENTIFIER`('abs')(c1) FROM VALUES(-1) AS T(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`IDENTIFIER`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "`IDENTIFIER`('abs')" + } ] +} + + +-- !query +CREATE TABLE IDENTIFIER(1)(c1 INT) USING csv +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.WRONG_TYPE", + "sqlState" : "42601", + "messageParameters" : { + "dataType" : "int", + "expr" : "1", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 25, + "fragment" : "1" + } ] +} + + +-- !query +CREATE TABLE IDENTIFIER('a.b.c')(c1 INT) USING csv +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +CREATE VIEW IDENTIFIER('a.b.c')(c1) AS VALUES(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +DROP TABLE IDENTIFIER('a.b.c') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +DROP VIEW IDENTIFIER('a.b.c') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +COMMENT ON TABLE IDENTIFIER('a.b.c.d') IS 'hello' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`.`c`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +VALUES(IDENTIFIER(1)()) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.WRONG_TYPE", + "sqlState" : "42601", + "messageParameters" : { + "dataType" : "int", + "expr" : "1", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 19, + "fragment" : "1" + } ] +} + + +-- !query +VALUES(IDENTIFIER('a.b.c.d')()) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", + "sqlState" : "42601", + "messageParameters" : { + "identifier" : "`a`.`b`.`c`.`d`", + "limit" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "IDENTIFIER('a.b.c.d')()" + } ] +} + + +-- !query +CREATE TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX.CREATE_TEMP_FUNC_WITH_DATABASE", + "sqlState" : "42000", + "messageParameters" : { + "database" : "`default`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 108, + "fragment" : "CREATE TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg'" + } ] +} + + +-- !query +DROP TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX.MULTI_PART_NAME", + "sqlState" : "42000", + "messageParameters" : { + "name" : "`default`.`myDoubleAvg`", + "statement" : "DROP TEMPORARY FUNCTION" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 63, + "fragment" : "DROP TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg')" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW IDENTIFIER('default.v')(c1) AS VALUES(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "TEMP_VIEW_NAME_TOO_MANY_NAME_PARTS", + "sqlState" : "428EK", + "messageParameters" : { + "actualName" : "`default`.`v`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "CREATE TEMPORARY VIEW IDENTIFIER('default.v')(c1) AS VALUES(1)" + } ] +} + + +-- !query +create temporary view identifier('v1') as (select my_col from (values (1), (2), (1) as (my_col)) group by 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +cache table identifier('t1') as (select my_col from (values (1), (2), (1) as (my_col)) group by 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table identifier('t2') using csv as (select my_col from (values (1), (2), (1) as (my_col)) group by 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into identifier('t2') select my_col from (values (3) as (my_col)) group by 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop view v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE agg = 'max' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE col = 'c1' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE tab = 'T' +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), + T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) +SELECT IDENTIFIER(agg)(IDENTIFIER(col)) FROM IDENTIFIER(tab) +-- !query schema +struct +-- !query output +c + + +-- !query +WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), + T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) +SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('T') +-- !query schema +struct +-- !query output +c + + +-- !query +WITH ABC(c1, c2) AS (VALUES(1, 2), (2, 3)) +SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('A' || 'BC') +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT row_number() OVER IDENTIFIER('x.win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''x.win''", + "hint" : "" + } +} + + +-- !query +SELECT T1.c1 FROM VALUES(1) AS T1(c1) JOIN VALUES(1) AS T2(c1) USING (IDENTIFIER('c1')) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT IDENTIFIER('t').c1 FROM VALUES(1) AS T(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t`", + "proposal" : "`c1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "IDENTIFIER('t')" + } ] +} + + +-- !query +SELECT map('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''a''", + "hint" : "" + } +} + + +-- !query +SELECT named_struct('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''a''", + "hint" : "" + } +} + + +-- !query +SELECT * FROM s.IDENTIFIER('tab') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM IDENTIFIER('s').IDENTIFIER('tab') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'IDENTIFIER'", + "hint" : "" + } +} + + +-- !query +SELECT * FROM IDENTIFIER('s').tab +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'IDENTIFIER'", + "hint" : "" + } +} + + +-- !query +SELECT row_number() OVER IDENTIFIER('win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''win''", + "hint" : "" + } +} + + +-- !query +SELECT row_number() OVER win FROM VALUES(1) AS T(c1) WINDOW IDENTIFIER('win') AS (ORDER BY c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing 'AS'" + } +} + + +-- !query +SELECT 1 AS IDENTIFIER('col1') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT my_table.* FROM VALUES (1, 2) AS IDENTIFIER('my_table')(IDENTIFIER('c1'), IDENTIFIER('c2')) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''my_table''", + "hint" : "" + } +} + + +-- !query +WITH identifier('v')(identifier('c1')) AS (VALUES(1)) (SELECT c1 FROM v) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''v''", + "hint" : "" + } +} + + +-- !query +CREATE OR REPLACE VIEW v(IDENTIFIER('c1')) AS VALUES(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT c1 FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`v`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 16, + "fragment" : "v" + } ] +} + + +-- !query +CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +INSERT INTO tab(IDENTIFIER('c1')) VALUES(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing ')'" + } +} + + +-- !query +SELECT c1 FROM tab +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 18, + "fragment" : "tab" + } ] +} + + +-- !query +ALTER TABLE IDENTIFIER('tab') RENAME COLUMN IDENTIFIER('c1') TO IDENTIFIER('col1') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT col1 FROM tab +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 18, + "stopIndex" : 20, + "fragment" : "tab" + } ] +} + + +-- !query +ALTER TABLE IDENTIFIER('tab') ADD COLUMN IDENTIFIER('c2') INT +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT c2 FROM tab +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 18, + "fragment" : "tab" + } ] +} + + +-- !query +ALTER TABLE IDENTIFIER('tab') DROP COLUMN IDENTIFIER('c2') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +ALTER TABLE IDENTIFIER('tab') RENAME TO IDENTIFIER('tab_renamed') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM tab_renamed +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`tab_renamed`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 25, + "fragment" : "tab_renamed" + } ] +} + + +-- !query +DROP TABLE IF EXISTS tab_renamed +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS tab +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE test_col_with_dot(IDENTIFIER('`col.with.dot`') INT) USING CSV +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE IF EXISTS test_col_with_dot +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM VALUES (1, 2) AS IDENTIFIER('schema.table')(c1, c2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''schema.table''", + "hint" : "" + } +} + + +-- !query +SELECT 1 AS IDENTIFIER('col1.col2') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +CREATE SCHEMA identifier_clause_test_schema +-- !query schema +struct<> +-- !query output + + + +-- !query +USE identifier_clause_test_schema +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE test_show(c1 INT, c2 STRING) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW VIEWS IN IDENTIFIER('identifier_clause_test_schema') +-- !query schema +struct +-- !query output + + + +-- !query +SHOW PARTITIONS IDENTIFIER('test_show') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_PARTITION_OPERATION.PARTITION_SCHEMA_IS_EMPTY", + "sqlState" : "42601", + "messageParameters" : { + "name" : "`spark_catalog`.`identifier_clause_test_schema`.`test_show`" + } +} + + +-- !query +SHOW CREATE TABLE IDENTIFIER('test_show') +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.identifier_clause_test_schema.test_show ( + c1 INT, + c2 STRING) +USING CSV + + +-- !query +DROP TABLE test_show +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE test_desc(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE TABLE IDENTIFIER('test_desc') +-- !query schema +struct +-- !query output +c1 int + + +-- !query +DESCRIBE FORMATTED IDENTIFIER('test_desc') +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database identifier_clause_test_schema +Table test_desc +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider CSV +Location [not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/test_desc + + +-- !query +DESCRIBE EXTENDED IDENTIFIER('test_desc') +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database identifier_clause_test_schema +Table test_desc +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider CSV +Location [not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/test_desc + + +-- !query +DESC IDENTIFIER('test_desc') +-- !query schema +struct +-- !query output +c1 int + + +-- !query +DROP TABLE test_desc +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE test_comment(c1 INT, c2 STRING) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +COMMENT ON TABLE IDENTIFIER('test_comment') IS 'table comment' +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER TABLE test_comment ALTER COLUMN IDENTIFIER('c1') COMMENT 'column comment' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE test_comment +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE identifier_clause_test_schema.test_table(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +ANALYZE TABLE IDENTIFIER('identifier_clause_test_schema.test_table') COMPUTE STATISTICS +-- !query schema +struct<> +-- !query output + + + +-- !query +REFRESH TABLE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query schema +struct +-- !query output +c1 int + + +-- !query +SHOW COLUMNS FROM IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query schema +struct +-- !query output +c1 + + +-- !query +DROP TABLE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE IDENTIFIER('my_var') = 'value' +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VAR IDENTIFIER('my_var') = 'new_value' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing EQ" + } +} + + +-- !query +SELECT IDENTIFIER('my_var') +-- !query schema +struct +-- !query output +value + + +-- !query +DROP TEMPORARY VARIABLE IDENTIFIER('my_var') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY FUNCTION test_udf(IDENTIFIER('param1') INT, IDENTIFIER('param2') STRING) +RETURNS INT +RETURN IDENTIFIER('param1') + length(IDENTIFIER('param2')) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT test_udf(5, 'hello') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`test_udf`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`identifier_clause_test_schema`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "test_udf(5, 'hello')" + } ] +} + + +-- !query +DROP TEMPORARY FUNCTION test_udf +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.analysis.NoSuchTempFunctionException +{ + "errorClass" : "ROUTINE_NOT_FOUND", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`test_udf`" + } +} + + +-- !query +CREATE TEMPORARY FUNCTION test_table_udf(IDENTIFIER('input_val') INT) +RETURNS TABLE(IDENTIFIER('col1') INT, IDENTIFIER('col2') STRING) +RETURN SELECT IDENTIFIER('input_val'), 'result' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM test_table_udf(42) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVABLE_TABLE_VALUED_FUNCTION", + "sqlState" : "42883", + "messageParameters" : { + "name" : "`test_table_udf`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 32, + "fragment" : "test_table_udf(42)" + } ] +} + + +-- !query +DROP TEMPORARY FUNCTION test_table_udf +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.analysis.NoSuchTempFunctionException +{ + "errorClass" : "ROUTINE_NOT_FOUND", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`test_table_udf`" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:tab \'b\').c1 FROM VALUES(1) AS tab(c1)' USING 'ta' AS tab +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_BASE_FIELD_TYPE", + "sqlState" : "42000", + "messageParameters" : { + "base" : "\"variablereference(system.session.tab='T')\"", + "other" : "\"STRING\"" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col1 ''.c2'') FROM VALUES(named_struct(''c2'', 42)) AS T(c1)' + USING 'c1' AS col1 +-- !query schema +struct +-- !query output +42 + + +-- !query +CREATE TABLE integration_test(c1 INT, c2 STRING) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO integration_test VALUES (1, 'a'), (2, 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''1''), IDENTIFIER(:prefix ''2'') FROM integration_test ORDER BY ALL' + USING 'c' AS prefix +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM integration_test WHERE IDENTIFIER(:col) = :val' + USING 'c1' AS col, 1 AS val +-- !query schema +struct +-- !query output +1 a + + +-- !query +CREATE TABLE integration_test2(c1 INT, c3 STRING) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO integration_test2 VALUES (1, 'x'), (2, 'y') +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT t1.*, t2.* FROM IDENTIFIER(:t1) t1 JOIN IDENTIFIER(:t2) t2 USING (IDENTIFIER(:col)) ORDER BY ALL' + USING 'integration_test' AS t1, 'integration_test2' AS t2, 'c1' AS col +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 103, + "fragment" : "SELECT t1.*, t2.* FROM IDENTIFIER(:t1) t1 JOIN IDENTIFIER(:t2) t2 USING (IDENTIFIER(:col)) ORDER BY ALL" + } ] +} + + +-- !query +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:col2), row_number() OVER (PARTITION BY IDENTIFIER(:part) ORDER BY IDENTIFIER(:ord)) as rn FROM integration_test' + USING 'c1' AS col1, 'c2' AS col2, 'c2' AS part, 'c1' AS ord +-- !query schema +struct +-- !query output +1 a 1 +2 b 1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''2''), IDENTIFIER(:agg)(IDENTIFIER(:col)) FROM integration_test GROUP BY IDENTIFIER(:prefix ''2'') ORDER BY ALL' + USING 'c' AS prefix, 'count' AS agg, 'c1' AS col +-- !query schema +struct +-- !query output +a 1 +b 1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM integration_test ORDER BY IDENTIFIER(:col1) DESC, IDENTIFIER(:col2)' + USING 'c1' AS col1, 'c2' AS col2 +-- !query schema +struct +-- !query output +2 b +1 a + + +-- !query +EXECUTE IMMEDIATE 'INSERT INTO integration_test(IDENTIFIER(:col1), IDENTIFIER(:col2)) VALUES (:val1, :val2)' + USING 'c1' AS col1, 'c2' AS col2, 3 AS val1, 'c' AS val2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing ')'" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "INSERT INTO integration_test(IDENTIFIER(:col1), IDENTIFIER(:col2)) VALUES (:val1, :val2)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(concat(:schema, ''.'', :table, ''.c1'')) FROM VALUES(named_struct(''c1'', 100)) AS IDENTIFIER(:alias)(IDENTIFIER(:schema ''.'' :table))' + USING 'identifier_clause_test_schema' AS schema, 'my_table' AS table, 't' AS alias +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "':'", + "hint" : ": extra input ':'" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 145, + "fragment" : "SELECT IDENTIFIER(concat(:schema, '.', :table, '.c1')) FROM VALUES(named_struct('c1', 100)) AS IDENTIFIER(:alias)(IDENTIFIER(:schema '.' :table))" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'WITH IDENTIFIER(:cte_name)(c1) AS (VALUES(1)) SELECT c1 FROM IDENTIFIER(:cte_name)' + USING 'my_cte' AS cte_name +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "':'", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 82, + "fragment" : "WITH IDENTIFIER(:cte_name)(c1) AS (VALUES(1)) SELECT c1 FROM IDENTIFIER(:cte_name)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'CREATE OR REPLACE TEMPORARY VIEW IDENTIFIER(:view_name)(IDENTIFIER(:col_name)) AS VALUES(1)' + USING 'test_view' AS view_name, 'test_col' AS col_name +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 91, + "fragment" : "CREATE OR REPLACE TEMPORARY VIEW IDENTIFIER(:view_name)(IDENTIFIER(:col_name)) AS VALUES(1)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col) FROM IDENTIFIER(:view)' + USING 'test_col' AS col, 'test_view' AS view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`test_view`" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 46, + "fragment" : "IDENTIFIER(:view)" + } ] +} + + +-- !query +DROP VIEW test_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`spark_catalog`.`identifier_clause_test_schema`.`test_view`" + } +} + + +-- !query +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) ADD COLUMN IDENTIFIER(:new_col) INT' + USING 'integration_test' AS tab, 'c4' AS new_col +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 64, + "fragment" : "ALTER TABLE IDENTIFIER(:tab) ADD COLUMN IDENTIFIER(:new_col) INT" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) RENAME COLUMN IDENTIFIER(:old_col) TO IDENTIFIER(:new_col)' + USING 'integration_test' AS tab, 'c4' AS old_col, 'c5' AS new_col +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 87, + "fragment" : "ALTER TABLE IDENTIFIER(:tab) RENAME COLUMN IDENTIFIER(:old_col) TO IDENTIFIER(:new_col)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT map(:key, :val).IDENTIFIER(:key) AS result' + USING 'mykey' AS key, 42 AS val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "':'", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 49, + "fragment" : "SELECT map(:key, :val).IDENTIFIER(:key) AS result" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:alias ''.c1'') FROM integration_test AS IDENTIFIER(:alias) ORDER BY ALL' + USING 't' AS alias +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "':'", + "hint" : ": extra input ':'" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "SELECT IDENTIFIER(:alias '.c1') FROM integration_test AS IDENTIFIER(:alias) ORDER BY ALL" + } ] +} + + +-- !query +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:p ''2'') FROM IDENTIFIER(:schema ''.'' :tab) WHERE IDENTIFIER(:col1) > 0 ORDER BY IDENTIFIER(:p ''1'')' + USING 'c1' AS col1, 'c' AS p, 'identifier_clause_test_schema' AS schema, 'integration_test' AS tab +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) WHERE IDENTIFIER(concat(:tab_alias, ''.c1'')) > 0 ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table, 'integration_test' AS tab_alias +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 AS IDENTIFIER(:schema ''.'' :col)' + USING 'identifier_clause_test_schema' AS schema, 'col1' AS col +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 40, + "fragment" : "SELECT 1 AS IDENTIFIER(:schema '.' :col)" + } ] +} + + +-- !query +DROP TABLE integration_test +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE integration_test2 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE lateral_test(arr ARRAY) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO lateral_test VALUES (array(1, 2, 3)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM lateral_test LATERAL VIEW explode(arr) IDENTIFIER('tbl') AS IDENTIFIER('col') ORDER BY ALL +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM lateral_test LATERAL VIEW OUTER explode(arr) IDENTIFIER('my_table') AS IDENTIFIER('my_col') ORDER BY ALL +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE lateral_test +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE unpivot_test(id INT, a INT, b INT, c INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO unpivot_test VALUES (1, 10, 20, 30) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM unpivot_test UNPIVOT (val FOR col IN (a AS IDENTIFIER('col_a'), b AS IDENTIFIER('col_b'))) ORDER BY ALL +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT * FROM unpivot_test UNPIVOT ((v1, v2) FOR col IN ((a, b) AS IDENTIFIER('cols_ab'), (b, c) AS IDENTIFIER('cols_bc'))) ORDER BY ALL +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE unpivot_test +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT :IDENTIFIER('param1') FROM VALUES(1) AS T(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''param1''", + "hint" : "" + } +} + + +-- !query +CREATE TABLE hint_test(c1 INT, c2 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO hint_test VALUES (1, 2), (3, 4) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT /*+ IDENTIFIER('BROADCAST')(hint_test) */ * FROM hint_test +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT /*+ IDENTIFIER('MERGE')(hint_test) */ * FROM hint_test +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE hint_test +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW IDENTIFIER('USER') FUNCTIONS +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT EXTRACT(IDENTIFIER('YEAR') FROM DATE'2024-01-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT TIMESTAMPADD(IDENTIFIER('YEAR'), 1, DATE'2024-01-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`TIMESTAMPADD`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`identifier_clause_test_schema`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 60, + "fragment" : "TIMESTAMPADD(IDENTIFIER('YEAR'), 1, DATE'2024-01-15')" + } ] +} + + +-- !query +DROP SCHEMA identifier_clause_test_schema +-- !query schema +struct<> +-- !query output + diff --git a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out index 596745b4ba5d..4a0f9abe5639 100644 --- a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out @@ -118,9 +118,11 @@ struct<> -- !query output org.apache.spark.SparkUnsupportedOperationException { - "errorClass" : "_LEGACY_ERROR_TEMP_2096", + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", "messageParameters" : { - "ddl" : "UPDATE TABLE" + "operation" : "UPDATE TABLE", + "tableName" : "`spark_catalog`.`s`.`tab`" } } @@ -133,9 +135,11 @@ struct<> -- !query output org.apache.spark.SparkUnsupportedOperationException { - "errorClass" : "_LEGACY_ERROR_TEMP_2096", + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", "messageParameters" : { - "ddl" : "MERGE INTO TABLE" + "operation" : "MERGE INTO TABLE", + "tableName" : "`spark_catalog`.`s`.`tab`" } } @@ -980,7 +984,8 @@ org.apache.spark.sql.AnalysisException "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", "sqlState" : "42601", "messageParameters" : { - "identifier" : "`a`.`b`.`c`.`d`" + "identifier" : "`a`.`b`.`c`.`d`", + "limit" : "2" }, "queryContext" : [ { "objectType" : "", @@ -1175,29 +1180,28 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "PARSE_SYNTAX_ERROR", + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", "sqlState" : "42601", "messageParameters" : { - "error" : "''x.win''", - "hint" : "" - } + "identifier" : "`x`.`win`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 44, + "fragment" : "IDENTIFIER('x.win')" + } ] } -- !query SELECT T1.c1 FROM VALUES(1) AS T1(c1) JOIN VALUES(1) AS T2(c1) USING (IDENTIFIER('c1')) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'('", - "hint" : "" - } -} +1 -- !query @@ -1226,33 +1230,17 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT map('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "''a''", - "hint" : "" - } -} +1 -- !query SELECT named_struct('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "''a''", - "hint" : "" - } -} +1 -- !query @@ -1260,12 +1248,12 @@ SELECT * FROM s.IDENTIFIER('tab') -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "INVALID_SQL_SYNTAX.INVALID_TABLE_VALUED_FUNC_NAME", - "sqlState" : "42000", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "funcName" : "`s`.`IDENTIFIER`" + "relationName" : "`s`.`tab`" }, "queryContext" : [ { "objectType" : "", @@ -1282,14 +1270,20 @@ SELECT * FROM IDENTIFIER('s').IDENTIFIER('tab') -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "error" : "'.'", - "hint" : "" - } + "relationName" : "`s`.`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 47, + "fragment" : "IDENTIFIER('s').IDENTIFIER('tab')" + } ] } @@ -1298,103 +1292,1049 @@ SELECT * FROM IDENTIFIER('s').tab -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "error" : "'.'", - "hint" : "" - } + "relationName" : "`s`.`tab`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 33, + "fragment" : "IDENTIFIER('s').tab" + } ] } -- !query SELECT row_number() OVER IDENTIFIER('win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1) -- !query schema +struct +-- !query output +1 + + +-- !query +SELECT row_number() OVER win FROM VALUES(1) AS T(c1) WINDOW IDENTIFIER('win') AS (ORDER BY c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT 1 AS IDENTIFIER('col1') +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT my_table.* FROM VALUES (1, 2) AS IDENTIFIER('my_table')(IDENTIFIER('c1'), IDENTIFIER('c2')) +-- !query schema +struct +-- !query output +1 2 + + +-- !query +WITH identifier('v')(identifier('c1')) AS (VALUES(1)) (SELECT c1 FROM v) +-- !query schema +struct +-- !query output +1 + + +-- !query +CREATE OR REPLACE VIEW v(IDENTIFIER('c1')) AS VALUES(1) +-- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException + + + +-- !query +SELECT c1 FROM v +-- !query schema +struct +-- !query output +1 + + +-- !query +CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO tab(IDENTIFIER('c1')) VALUES(1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT c1 FROM tab +-- !query schema +struct +-- !query output +1 + + +-- !query +ALTER TABLE IDENTIFIER('tab') RENAME COLUMN IDENTIFIER('c1') TO IDENTIFIER('col1') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", "messageParameters" : { - "error" : "''win''", - "hint" : "" + "operation" : "RENAME COLUMN", + "tableName" : "`spark_catalog`.`default`.`tab`" } } -- !query -SELECT row_number() OVER win FROM VALUES(1) AS T(c1) WINDOW IDENTIFIER('win') AS (ORDER BY c1) +SELECT col1 FROM tab -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", "messageParameters" : { - "error" : "'WINDOW'", - "hint" : "" - } + "objectName" : "`col1`", + "proposal" : "`c1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 11, + "fragment" : "col1" + } ] } -- !query -WITH identifier('v')(identifier('c1')) AS (VALUES(1)) (SELECT c1 FROM v) +ALTER TABLE IDENTIFIER('tab') ADD COLUMN IDENTIFIER('c2') INT -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException + + + +-- !query +SELECT c2 FROM tab +-- !query schema +struct +-- !query output +NULL + + +-- !query +ALTER TABLE IDENTIFIER('tab') DROP COLUMN IDENTIFIER('c2') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", "messageParameters" : { - "error" : "''v''", - "hint" : "" + "operation" : "DROP COLUMN", + "tableName" : "`spark_catalog`.`default`.`tab`" } } -- !query -INSERT INTO tab(IDENTIFIER('c1')) VALUES(1) +ALTER TABLE IDENTIFIER('tab') RENAME TO IDENTIFIER('tab_renamed') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM tab_renamed +-- !query schema +struct +-- !query output +1 NULL + + +-- !query +DROP TABLE IF EXISTS tab_renamed +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS tab +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE test_col_with_dot(IDENTIFIER('`col.with.dot`') INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS test_col_with_dot +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM VALUES (1, 2) AS IDENTIFIER('schema.table')(c1, c2) -- !query schema struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "PARSE_SYNTAX_ERROR", + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", "sqlState" : "42601", "messageParameters" : { - "error" : "'('", - "hint" : ": missing ')'" - } + "identifier" : "`schema`.`table`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 65, + "fragment" : "VALUES (1, 2) AS IDENTIFIER('schema.table')(c1, c2)" + } ] } -- !query -CREATE OR REPLACE VIEW v(IDENTIFIER('c1')) AS VALUES(1) +SELECT 1 AS IDENTIFIER('col1.col2') -- !query schema struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "PARSE_SYNTAX_ERROR", + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", "sqlState" : "42601", "messageParameters" : { - "error" : "'('", - "hint" : "" - } + "identifier" : "`col1`.`col2`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "1 AS IDENTIFIER('col1.col2')" + } ] } -- !query -CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV +CREATE SCHEMA identifier_clause_test_schema -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException + + + +-- !query +USE identifier_clause_test_schema +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE test_show(c1 INT, c2 STRING) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW VIEWS IN IDENTIFIER('identifier_clause_test_schema') +-- !query schema +struct +-- !query output + + + +-- !query +SHOW PARTITIONS IDENTIFIER('test_show') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_PARTITION_OPERATION.PARTITION_SCHEMA_IS_EMPTY", + "sqlState" : "42601", + "messageParameters" : { + "name" : "`spark_catalog`.`identifier_clause_test_schema`.`test_show`" + } +} + + +-- !query +SHOW CREATE TABLE IDENTIFIER('test_show') +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.identifier_clause_test_schema.test_show ( + c1 INT, + c2 STRING) +USING CSV + + +-- !query +DROP TABLE test_show +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE test_desc(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE TABLE IDENTIFIER('test_desc') +-- !query schema +struct +-- !query output +c1 int + + +-- !query +DESCRIBE FORMATTED IDENTIFIER('test_desc') +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database identifier_clause_test_schema +Table test_desc +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider CSV +Location [not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/test_desc + + +-- !query +DESCRIBE EXTENDED IDENTIFIER('test_desc') +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database identifier_clause_test_schema +Table test_desc +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider CSV +Location [not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/test_desc + + +-- !query +DESC IDENTIFIER('test_desc') +-- !query schema +struct +-- !query output +c1 int + + +-- !query +DROP TABLE test_desc +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE test_comment(c1 INT, c2 STRING) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +COMMENT ON TABLE IDENTIFIER('test_comment') IS 'table comment' +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER TABLE test_comment ALTER COLUMN IDENTIFIER('c1') COMMENT 'column comment' +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE test_comment +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE identifier_clause_test_schema.test_table(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +ANALYZE TABLE IDENTIFIER('identifier_clause_test_schema.test_table') COMPUTE STATISTICS +-- !query schema +struct<> +-- !query output + + + +-- !query +REFRESH TABLE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query schema +struct +-- !query output +c1 int + + +-- !query +SHOW COLUMNS FROM IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query schema +struct +-- !query output +c1 + + +-- !query +DROP TABLE IDENTIFIER('identifier_clause_test_schema.test_table') +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE IDENTIFIER('my_var') = 'value' +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VAR IDENTIFIER('my_var') = 'new_value' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT IDENTIFIER('my_var') +-- !query schema +struct +-- !query output +new_value + + +-- !query +DROP TEMPORARY VARIABLE IDENTIFIER('my_var') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY FUNCTION test_udf(IDENTIFIER('param1') INT, IDENTIFIER('param2') STRING) +RETURNS INT +RETURN IDENTIFIER('param1') + length(IDENTIFIER('param2')) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT test_udf(5, 'hello') +-- !query schema +struct +-- !query output +10 + + +-- !query +DROP TEMPORARY FUNCTION test_udf +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY FUNCTION test_table_udf(IDENTIFIER('input_val') INT) +RETURNS TABLE(IDENTIFIER('col1') INT, IDENTIFIER('col2') STRING) +RETURN SELECT IDENTIFIER('input_val'), 'result' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM test_table_udf(42) +-- !query schema +struct +-- !query output +42 result + + +-- !query +DROP TEMPORARY FUNCTION test_table_udf +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:tab \'b\').c1 FROM VALUES(1) AS tab(c1)' USING 'ta' AS tab +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_BASE_FIELD_TYPE", + "sqlState" : "42000", + "messageParameters" : { + "base" : "\"variablereference(system.session.tab='T')\"", + "other" : "\"STRING\"" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col1 ''.c2'') FROM VALUES(named_struct(''c2'', 42)) AS T(c1)' + USING 'c1' AS col1 +-- !query schema +struct +-- !query output +42 + + +-- !query +CREATE TABLE integration_test(c1 INT, c2 STRING) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO integration_test VALUES (1, 'a'), (2, 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''1''), IDENTIFIER(:prefix ''2'') FROM integration_test ORDER BY ALL' + USING 'c' AS prefix +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM integration_test WHERE IDENTIFIER(:col) = :val' + USING 'c1' AS col, 1 AS val +-- !query schema +struct +-- !query output +1 a + + +-- !query +CREATE TABLE integration_test2(c1 INT, c3 STRING) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO integration_test2 VALUES (1, 'x'), (2, 'y') +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT t1.*, t2.* FROM IDENTIFIER(:t1) t1 JOIN IDENTIFIER(:t2) t2 USING (IDENTIFIER(:col)) ORDER BY ALL' + USING 'integration_test' AS t1, 'integration_test2' AS t2, 'c1' AS col +-- !query schema +struct +-- !query output +1 a 1 x +2 b 2 y + + +-- !query +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:col2), row_number() OVER (PARTITION BY IDENTIFIER(:part) ORDER BY IDENTIFIER(:ord)) as rn FROM integration_test' + USING 'c1' AS col1, 'c2' AS col2, 'c2' AS part, 'c1' AS ord +-- !query schema +struct +-- !query output +1 a 1 +2 b 1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:prefix ''2''), IDENTIFIER(:agg)(IDENTIFIER(:col)) FROM integration_test GROUP BY IDENTIFIER(:prefix ''2'') ORDER BY ALL' + USING 'c' AS prefix, 'count' AS agg, 'c1' AS col +-- !query schema +struct +-- !query output +a 1 +b 1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM integration_test ORDER BY IDENTIFIER(:col1) DESC, IDENTIFIER(:col2)' + USING 'c1' AS col1, 'c2' AS col2 +-- !query schema +struct +-- !query output +2 b +1 a + + +-- !query +EXECUTE IMMEDIATE 'INSERT INTO integration_test(IDENTIFIER(:col1), IDENTIFIER(:col2)) VALUES (:val1, :val2)' + USING 'c1' AS col1, 'c2' AS col2, 3 AS val1, 'c' AS val2 +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(concat(:schema, ''.'', :table, ''.c1'')) FROM VALUES(named_struct(''c1'', 100)) AS IDENTIFIER(:alias)(IDENTIFIER(:schema ''.'' :table))' + USING 'identifier_clause_test_schema' AS schema, 'my_table' AS table, 't' AS alias +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", + "sqlState" : "42601", + "messageParameters" : { + "identifier" : "`identifier_clause_test_schema`.`my_table`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 115, + "stopIndex" : 144, + "fragment" : "IDENTIFIER(:schema '.' :table)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'WITH IDENTIFIER(:cte_name)(c1) AS (VALUES(1)) SELECT c1 FROM IDENTIFIER(:cte_name)' + USING 'my_cte' AS cte_name +-- !query schema +struct +-- !query output +1 + + +-- !query +EXECUTE IMMEDIATE 'CREATE OR REPLACE TEMPORARY VIEW IDENTIFIER(:view_name)(IDENTIFIER(:col_name)) AS VALUES(1)' + USING 'test_view' AS view_name, 'test_col' AS col_name +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:col) FROM IDENTIFIER(:view)' + USING 'test_col' AS col, 'test_view' AS view +-- !query schema +struct +-- !query output +1 + + +-- !query +DROP VIEW test_view +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) ADD COLUMN IDENTIFIER(:new_col) INT' + USING 'integration_test' AS tab, 'c4' AS new_col +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'ALTER TABLE IDENTIFIER(:tab) RENAME COLUMN IDENTIFIER(:old_col) TO IDENTIFIER(:new_col)' + USING 'integration_test' AS tab, 'c4' AS old_col, 'c5' AS new_col +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "RENAME COLUMN", + "tableName" : "`spark_catalog`.`identifier_clause_test_schema`.`integration_test`" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT map(:key, :val).IDENTIFIER(:key) AS result' + USING 'mykey' AS key, 42 AS val +-- !query schema +struct +-- !query output +42 + + +-- !query +EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:alias ''.c1'') FROM integration_test AS IDENTIFIER(:alias) ORDER BY ALL' + USING 't' AS alias +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t`.`c1`", + "proposal" : "`IDENTIFIER('t')`.`c1`, `IDENTIFIER('t')`.`c2`, `IDENTIFIER('t')`.`c4`" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "IDENTIFIER(:alias '.c1')" + } ] +} + + +-- !query +EXECUTE IMMEDIATE + 'SELECT IDENTIFIER(:col1), IDENTIFIER(:p ''2'') FROM IDENTIFIER(:schema ''.'' :tab) WHERE IDENTIFIER(:col1) > 0 ORDER BY IDENTIFIER(:p ''1'')' + USING 'c1' AS col1, 'c' AS p, 'identifier_clause_test_schema' AS schema, 'integration_test' AS tab +-- !query schema +struct +-- !query output +1 a +2 b +3 c + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM IDENTIFIER(:schema ''.'' :table) WHERE IDENTIFIER(concat(:tab_alias, ''.c1'')) > 0 ORDER BY ALL' + USING 'identifier_clause_test_schema' AS schema, 'integration_test' AS table, 'integration_test' AS tab_alias +-- !query schema +struct +-- !query output +1 a NULL +2 b NULL +3 c NULL + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 AS IDENTIFIER(:schema ''.'' :col)' + USING 'identifier_clause_test_schema' AS schema, 'col1' AS col +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", + "sqlState" : "42601", + "messageParameters" : { + "identifier" : "`identifier_clause_test_schema`.`col1`", + "limit" : "1" + }, + "queryContext" : [ { + "objectType" : "EXECUTE IMMEDIATE", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "1 AS IDENTIFIER(:schema '.' :col)" + } ] +} + + +-- !query +DROP TABLE integration_test +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE integration_test2 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE lateral_test(arr ARRAY) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO lateral_test VALUES (array(1, 2, 3)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM lateral_test LATERAL VIEW explode(arr) IDENTIFIER('tbl') AS IDENTIFIER('col') ORDER BY ALL +-- !query schema +struct,col:int> +-- !query output +[1,2,3] 1 +[1,2,3] 2 +[1,2,3] 3 + + +-- !query +SELECT * FROM lateral_test LATERAL VIEW OUTER explode(arr) IDENTIFIER('my_table') AS IDENTIFIER('my_col') ORDER BY ALL +-- !query schema +struct,my_col:int> +-- !query output +[1,2,3] 1 +[1,2,3] 2 +[1,2,3] 3 + + +-- !query +DROP TABLE lateral_test +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE unpivot_test(id INT, a INT, b INT, c INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO unpivot_test VALUES (1, 10, 20, 30) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM unpivot_test UNPIVOT (val FOR col IN (a AS IDENTIFIER('col_a'), b AS IDENTIFIER('col_b'))) ORDER BY ALL +-- !query schema +struct +-- !query output +1 30 col_a 10 +1 30 col_b 20 + + +-- !query +SELECT * FROM unpivot_test UNPIVOT ((v1, v2) FOR col IN ((a, b) AS IDENTIFIER('cols_ab'), (b, c) AS IDENTIFIER('cols_bc'))) ORDER BY ALL +-- !query schema +struct +-- !query output +1 cols_ab 10 20 +1 cols_bc 20 30 + + +-- !query +DROP TABLE unpivot_test +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT :IDENTIFIER('param1') FROM VALUES(1) AS T(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''param1''", + "hint" : "" + } +} + + +-- !query +CREATE TABLE hint_test(c1 INT, c2 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO hint_test VALUES (1, 2), (3, 4) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT /*+ IDENTIFIER('BROADCAST')(hint_test) */ * FROM hint_test +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT /*+ IDENTIFIER('MERGE')(hint_test) */ * FROM hint_test +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE hint_test +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW IDENTIFIER('USER') FUNCTIONS +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException { "errorClass" : "PARSE_SYNTAX_ERROR", "sqlState" : "42601", @@ -1403,3 +2343,50 @@ org.apache.spark.sql.catalyst.parser.ParseException "hint" : "" } } + + +-- !query +SELECT EXTRACT(IDENTIFIER('YEAR') FROM DATE'2024-01-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'FROM'", + "hint" : ": missing ')'" + } +} + + +-- !query +SELECT TIMESTAMPADD(IDENTIFIER('YEAR'), 1, DATE'2024-01-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`TIMESTAMPADD`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`identifier_clause_test_schema`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 60, + "fragment" : "TIMESTAMPADD(IDENTIFIER('YEAR'), 1, DATE'2024-01-15')" + } ] +} + + +-- !query +DROP SCHEMA identifier_clause_test_schema +-- !query schema +struct<> +-- !query output + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala index e30b48fdb176..40cfb54159d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala @@ -2374,4 +2374,76 @@ class ParametersSuite extends QueryTest with SharedSparkSession { expectedStopPos = Some(46) // End of "nonexistent_table" in inner query ) } + + test("detect unbound named parameter with empty map") { + // When sql() is called with empty map, parameter markers should still be detected + val exception = intercept[AnalysisException] { + spark.sql("SELECT :param", Map.empty[String, Any]) + } + checkError( + exception = exception, + condition = "UNBOUND_SQL_PARAMETER", + parameters = Map("name" -> "param"), + context = ExpectedContext( + fragment = ":param", + start = 7, + stop = 12)) + } + + test("detect unbound positional parameter with empty array") { + // When sql() is called with empty array, parameter markers should still be detected + val exception = intercept[AnalysisException] { + spark.sql("SELECT ?", Array.empty[Any]) + } + checkError( + exception = exception, + condition = "UNBOUND_SQL_PARAMETER", + parameters = Map("name" -> "_7"), + context = ExpectedContext( + fragment = "?", + start = 7, + stop = 7)) + } + + test("detect unbound named parameter with no arguments") { + val exception = intercept[AnalysisException] { + spark.sql("SELECT :param") + } + checkError( + exception = exception, + condition = "UNBOUND_SQL_PARAMETER", + parameters = Map("name" -> "param"), + context = ExpectedContext( + fragment = ":param", + start = 7, + stop = 12)) + } + + test("detect unbound positional parameter with no arguments") { + val exception = intercept[AnalysisException] { + spark.sql("SELECT ?") + } + checkError( + exception = exception, + condition = "UNBOUND_SQL_PARAMETER", + parameters = Map("name" -> "_7"), + context = ExpectedContext( + fragment = "?", + start = 7, + stop = 7)) + } + + test("empty map with no parameters - should succeed") { + // When there are no parameter markers, empty map should work fine + checkAnswer( + spark.sql("SELECT 1", Map.empty[String, Any]), + Row(1)) + } + + test("empty array with no parameters - should succeed") { + // When there are no parameter markers, empty array should work fine + checkAnswer( + spark.sql("SELECT 1", Array.empty[Any]), + Row(1)) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringLiteralCoalescingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringLiteralCoalescingSuite.scala index ea305f219aa5..e98cfc8bebcc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringLiteralCoalescingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringLiteralCoalescingSuite.scala @@ -867,4 +867,41 @@ class StringLiteralCoalescingSuite extends QueryTest with SharedSparkSession { Row("1a2b3c4") ) } + + test("parameter substitution with quote spacing - legacy consecutive string literals disabled") { + // With LEGACY_CONSECUTIVE_STRING_LITERALS enabled, '' would normally produce a single quote + // But with parameter substitution, 'literal':param should insert a space to prevent + // the closing quote and opening quote from being interpreted as an escape sequence + withSQLConf("spark.sql.legacy.consecutiveStringLiterals.enabled" -> "false") { + checkAnswer( + spark.sql("SELECT 'hello':p, 'hello''world'", Map("p" -> "world")), + // Space parameter separates literals, no singleton quote in middle + Row("helloworld", "hello'world") + ) + } + } + + test("parameter substitution with quote spacing - legacy consecutive string literals enabled") { + withSQLConf("spark.sql.legacy.consecutiveStringLiterals.enabled" -> "true") { + checkAnswer( + spark.sql("SELECT 'hello':p, 'hello''world'", Map("p" -> "world")), + // Space parameter separates literals, no singleton quote in middle + Row("helloworld", "helloworld") + ) + } + } + + // ======================================================================== + // Legacy Mode Tests - JSON Path Expressions vs Parameter Substitution + // ======================================================================== + test("JSON path expression - new mode with parameter substitution") { + // In new mode (constantsOnly=false), :name is a parameter marker when args provided + withSQLConf("spark.sql.legacy.parameterSubstitution.constantsOnly" -> "false") { + // The :name gets substituted with the parameter value + checkAnswer( + spark.sql("SELECT '{\"name\":\"joe\"}' :name", Map("name" -> "replaced")), + Row("{\"name\":\"joe\"}replaced") + ) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 21538ec8e44a..18a04cee7d92 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -2312,8 +2312,10 @@ class DataSourceV2SQLSuiteV1Filter exception = intercept[SparkUnsupportedOperationException] { sql(s"UPDATE $t SET name='Robert', age=32 WHERE p=1") }, - condition = "_LEGACY_ERROR_TEMP_2096", - parameters = Map("ddl" -> "UPDATE TABLE") + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + parameters = Map( + "tableName" -> "`testcat`.`ns1`.`ns2`.`tbl`", + "operation" -> "UPDATE TABLE") ) } } @@ -2418,8 +2420,10 @@ class DataSourceV2SQLSuiteV1Filter |WHEN MATCHED AND (target.p > 0) THEN UPDATE SET * |WHEN NOT MATCHED THEN INSERT *""".stripMargin) }, - condition = "_LEGACY_ERROR_TEMP_2096", - parameters = Map("ddl" -> "MERGE INTO TABLE")) + condition = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + parameters = Map( + "tableName" -> "`testcat`.`ns1`.`ns2`.`target`", + "operation" -> "MERGE INTO TABLE")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala index 9e16c5b2a3dd..45a656f7a6d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala @@ -720,7 +720,7 @@ class QueryCompilationErrorsSuite }, condition = "IDENTIFIER_TOO_MANY_NAME_PARTS", sqlState = "42601", - parameters = Map("identifier" -> "`db_name`.`schema_name`.`view_name`") + parameters = Map("identifier" -> "`db_name`.`schema_name`.`view_name`", "limit" -> "2") ) } @@ -741,7 +741,7 @@ class QueryCompilationErrorsSuite }, condition = "IDENTIFIER_TOO_MANY_NAME_PARTS", sqlState = "42601", - parameters = Map("identifier" -> "`db_name`.`schema_name`.`new_table_name`") + parameters = Map("identifier" -> "`db_name`.`schema_name`.`new_table_name`", "limit" -> "2") ) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala index e76a8556230a..3f25a1e139fa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala @@ -1163,7 +1163,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { checkError( exception = intercept[AnalysisException](testIdent.asTableIdentifier), condition = "IDENTIFIER_TOO_MANY_NAME_PARTS", - parameters = Map("identifier" -> "`a`.`b`.`c`") + parameters = Map("identifier" -> "`a`.`b`.`c`", "limit" -> "2") ) } @@ -1172,7 +1172,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { checkError( exception = intercept[AnalysisException](testIdent.asFunctionIdentifier), condition = "IDENTIFIER_TOO_MANY_NAME_PARTS", - parameters = Map("identifier" -> "`a`.`b`.`c`") + parameters = Map("identifier" -> "`a`.`b`.`c`", "limit" -> "2") ) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala index c6a48d98fa98..fd88559d4f98 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala @@ -3078,7 +3078,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel }, condition = "IDENTIFIER_TOO_MANY_NAME_PARTS", sqlState = "42601", - parameters = Map("identifier" -> "`db_name`.`schema_name`.`function_name`") + parameters = Map("identifier" -> "`db_name`.`schema_name`.`function_name`", "limit" -> "2") ) } finally { JdbcDialects.unregisterDialect(testH2Dialect) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingE2eSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingE2eSuite.scala index 02b4a19a44ad..9996bec44e23 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingE2eSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingE2eSuite.scala @@ -273,4 +273,66 @@ class SqlScriptingE2eSuite extends QueryTest with SharedSparkSession { condition = "INVALID_QUERY_MIXED_QUERY_PARAMETERS", parameters = Map()) } + + test("SQL Script labels with identifier") { + val sqlScript = + """ + |BEGIN + | IDENTIFIER('loop_label'): LOOP + | SELECT 1; + | LEAVE IDENTIFIER('loop_label'); + | END LOOP IDENTIFIER('loop_label'); + |END""".stripMargin + verifySqlScriptResult(sqlScript, Seq(Row(1))) + } + + test("SQL Script with labeled BEGIN/END block using identifier") { + val sqlScript = + """ + |BEGIN + | IDENTIFIER('block_label'): BEGIN + | DECLARE IDENTIFIER('x') INT DEFAULT 1; + | SELECT x; + | END IDENTIFIER('block_label'); + |END""".stripMargin + verifySqlScriptResult(sqlScript, Seq(Row(1))) + } + + test("WHILE loop with identifier label") { + val sqlScript = + """ + |BEGIN + | DECLARE counter INT DEFAULT 0; + | IDENTIFIER('while_label'): WHILE counter < 3 DO + | SET IDENTIFIER('counter') = counter + 1; + | END WHILE IDENTIFIER('while_label'); + | SELECT counter; + |END""".stripMargin + verifySqlScriptResult(sqlScript, Seq(Row(3))) + } + + test("REPEAT loop with identifier label") { + val sqlScript = + """ + |BEGIN + | DECLARE cnt INT DEFAULT 0; + | repeat_label: REPEAT + | SET cnt = cnt + 1; + | UNTIL cnt >= 2 + | END REPEAT IDENTIFIER('repeat_label'); + | SELECT cnt; + |END""".stripMargin + verifySqlScriptResult(sqlScript, Seq(Row(2))) + } + + test("FOR loop with identifier") { + val sqlScript = + """ + |BEGIN + | IDENTIFIER('for_label'): FOR IDENTIFIER('row') AS SELECT 1 AS c1 DO + | SELECT row.c1; + | END FOR for_label; + |END""".stripMargin + verifySqlScriptResult(sqlScript, Seq(Row(1))) + } }