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 212c80a3cb43..51c846f93c1e 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 @@ -497,7 +497,7 @@ class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with DataTypeE override def visitComplexColType(ctx: ComplexColTypeContext): StructField = withOrigin(ctx) { import ctx._ val structField = StructField( - name = errorCapturingIdentifier.getText, + name = getIdentifierText(errorCapturingIdentifier), dataType = typedVisit(dataType()), nullable = NULL == null) Option(commentSpec).map(visitCommentSpec).map(structField.withComment).getOrElse(structField) 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 f918232c42ac..7a9d40ab5c9d 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 @@ -1235,7 +1235,7 @@ class AstBuilder extends DataTypeAstBuilder if (pVal.DEFAULT != null) { throw QueryParsingErrors.defaultColumnReferencesNotAllowedInPartitionSpec(ctx) } - val name = pVal.identifier.getText + val name = getIdentifierText(pVal.identifier) val value = Option(pVal.constant).map(v => { visitStringConstant(v, legacyNullAsString, keepPartitionSpecAsString) }) @@ -1958,11 +1958,11 @@ class AstBuilder extends DataTypeAstBuilder .flatMap(_.namedExpression.asScala) .map(typedVisit[Expression]) val pivotColumn = if (ctx.pivotColumn.identifiers.size == 1) { - UnresolvedAttribute.quoted(ctx.pivotColumn.errorCapturingIdentifier.getText) + UnresolvedAttribute.quoted(getIdentifierText(ctx.pivotColumn.errorCapturingIdentifier)) } else { CreateStruct( ctx.pivotColumn.identifiers.asScala.map( - identifier => UnresolvedAttribute.quoted(identifier.getText)).toSeq) + identifier => UnresolvedAttribute.quoted(getIdentifierText(identifier))).toSeq) } val pivotValues = ctx.pivotValues.asScala.map(visitPivotValue) Pivot(None, pivotColumn, pivotValues.toSeq, aggregates, query) @@ -1974,7 +1974,7 @@ class AstBuilder extends DataTypeAstBuilder override def visitPivotValue(ctx: PivotValueContext): Expression = withOrigin(ctx) { val e = expression(ctx.expression) if (ctx.errorCapturingIdentifier != null) { - Alias(e, ctx.errorCapturingIdentifier.getText)() + Alias(e, getIdentifierText(ctx.errorCapturingIdentifier))() } else { e } @@ -2039,7 +2039,7 @@ class AstBuilder extends DataTypeAstBuilder // alias unpivot result if (ctx.errorCapturingIdentifier() != null) { - val alias = ctx.errorCapturingIdentifier().getText + val alias = getIdentifierText(ctx.errorCapturingIdentifier()) SubqueryAlias(alias, filtered) } else { filtered @@ -2541,7 +2541,7 @@ class AstBuilder extends DataTypeAstBuilder */ private def mayApplyAliasPlan(tableAlias: TableAliasContext, plan: LogicalPlan): LogicalPlan = { if (tableAlias.strictIdentifier != null) { - val alias = tableAlias.strictIdentifier.getText + val alias = getIdentifierText(tableAlias.strictIdentifier) if (tableAlias.identifierList != null) { val columnNames = visitIdentifierList(tableAlias.identifierList) SubqueryAlias(alias, UnresolvedSubqueryColumnAliases(columnNames, plan)) @@ -3229,7 +3229,7 @@ class AstBuilder extends DataTypeAstBuilder */ override def visitLambda(ctx: LambdaContext): Expression = withOrigin(ctx) { val arguments = ctx.identifier().asScala.map { name => - UnresolvedNamedLambdaVariable(UnresolvedAttribute.quoted(name.getText).nameParts) + UnresolvedNamedLambdaVariable(UnresolvedAttribute.quoted(getIdentifierText(name)).nameParts) } val function = expression(ctx.expression).transformUp { case a: UnresolvedAttribute => UnresolvedNamedLambdaVariable(a.nameParts) @@ -4261,7 +4261,7 @@ class AstBuilder extends DataTypeAstBuilder if (!SQLConf.get.objectLevelCollationsEnabled) { throw QueryCompilationErrors.objectLevelCollationsNotEnabledError() } - val collationName = ctx.identifier.getText + val collationName = getIdentifierText(ctx.identifier) CollationFactory.fetchCollation(collationName).collationName } @@ -4500,7 +4500,7 @@ class AstBuilder extends DataTypeAstBuilder def getFieldReference( ctx: ApplyTransformContext, arg: V2Expression): FieldReference = { - lazy val name: String = ctx.identifier.getText + lazy val name: String = getIdentifierText(ctx.identifier) arg match { case ref: FieldReference => ref @@ -4512,7 +4512,7 @@ class AstBuilder extends DataTypeAstBuilder def getSingleFieldReference( ctx: ApplyTransformContext, arguments: Seq[V2Expression]): FieldReference = { - lazy val name: String = ctx.identifier.getText + lazy val name: String = getIdentifierText(ctx.identifier) if (arguments.size > 1) { throw QueryParsingErrors.wrongNumberArgumentsForTransformError(name, arguments.size, ctx) } else if (arguments.isEmpty) { @@ -4797,7 +4797,7 @@ class AstBuilder extends DataTypeAstBuilder string(visitStringLit(c.outFmt))))) // Expected format: SEQUENCEFILE | TEXTFILE | RCFILE | ORC | PARQUET | AVRO case (c: GenericFileFormatContext, null) => - SerdeInfo(storedAs = Some(c.identifier.getText)) + SerdeInfo(storedAs = Some(getIdentifierText(c.identifier))) case (null, storageHandler) => invalidStatement("STORED BY", ctx) case _ => @@ -6433,7 +6433,7 @@ class AstBuilder extends DataTypeAstBuilder * }}} */ override def visitDropIndex(ctx: DropIndexContext): LogicalPlan = withOrigin(ctx) { - val indexName = ctx.identifier.getText + val indexName = getIdentifierText(ctx.identifier) DropIndex( createUnresolvedTable(ctx.identifierReference, "DROP INDEX"), indexName, @@ -6655,7 +6655,7 @@ class AstBuilder extends DataTypeAstBuilder target = None, excepts = ids.map(s => Seq(s)), replacements = None)) Project(projectList, left) }.getOrElse(Option(ctx.AS).map { _ => - SubqueryAlias(ctx.errorCapturingIdentifier().getText, left) + SubqueryAlias(getIdentifierText(ctx.errorCapturingIdentifier()), left) }.getOrElse(Option(ctx.whereClause).map { c => if (ctx.windowClause() != null) { throw QueryParsingErrors.windowClauseInPipeOperatorWhereClauseNotAllowedError(ctx) 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 f8f6e31be1bc..48b5065f59cc 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 @@ -187,7 +187,7 @@ class SparkSqlAstBuilder extends AstBuilder { (ident, _) => builder(ident)) } else if (ctx.errorCapturingIdentifier() != null) { // resolve immediately - builder.apply(Seq(ctx.errorCapturingIdentifier().getText)) + builder.apply(Seq(getIdentifierText(ctx.errorCapturingIdentifier()))) } else if (ctx.stringLit() != null) { // resolve immediately builder.apply(Seq(string(visitStringLit(ctx.stringLit())))) @@ -1308,7 +1308,7 @@ class SparkSqlAstBuilder extends AstBuilder { } else { DescribeColumn( relation, - UnresolvedAttribute(ctx.describeColName.nameParts.asScala.map(_.getText).toSeq), + UnresolvedAttribute(ctx.describeColName.nameParts.asScala.map(getIdentifierText).toSeq), isExtended) } } else { 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 index e53a8153e829..ce98652d3eba 100644 --- 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 @@ -2436,6 +2436,306 @@ DropTable false, false +- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.unpivot_test +-- !query +CREATE TABLE unpivot_alias_test(id INT, a INT, b INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`unpivot_alias_test`, false + + +-- !query +INSERT INTO unpivot_alias_test VALUES (1, 10, 20) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/unpivot_alias_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/unpivot_alias_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`unpivot_alias_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/unpivot_alias_test), [id, a, b] ++- Project [col1#x AS id#x, col2#x AS a#x, col3#x AS b#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +SELECT * FROM unpivot_alias_test UNPIVOT (val FOR col IN (a, b)) AS IDENTIFIER('unpivoted_result') 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_alias_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.unpivot_alias_test + + +-- !query +CREATE TABLE pivot_test(product STRING, quarter STRING, revenue INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`pivot_test`, false + + +-- !query +INSERT INTO pivot_test VALUES ('A', 'Q1', 100), ('A', 'Q2', 150), ('B', 'Q1', 200), ('B', 'Q2', 250) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/pivot_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/pivot_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`pivot_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/pivot_test), [product, quarter, revenue] ++- Project [col1#x AS product#x, col2#x AS quarter#x, col3#x AS revenue#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +SELECT * FROM pivot_test PIVOT (SUM(revenue) FOR IDENTIFIER('quarter') IN ('Q1', 'Q2')) ORDER BY product +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing 'IN'" + } +} + + +-- !query +SELECT * FROM pivot_test PIVOT (SUM(revenue) AS IDENTIFIER('total') FOR quarter IN ('Q1' AS IDENTIFIER('first_quarter'), 'Q2' AS IDENTIFIER('second_quarter'))) ORDER BY product +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing 'FOR'" + } +} + + +-- !query +DROP TABLE pivot_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.pivot_test + + +-- !query +SELECT transform(array(1, 2, 3), IDENTIFIER('x') -> x + 1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'IDENTIFIER'", + "hint" : "" + } +} + + +-- !query +SELECT * FROM VALUES(1, 2) AS T(c1, c2) |> AS IDENTIFIER('pipe_alias') |> SELECT c1, c2 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT c1, c2 FROM VALUES(1, 2) AS T(c1, c2) |> AS IDENTIFIER('my_result') |> SELECT * +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT CAST(named_struct('field1', 1, 'field2', 'hello') AS STRUCT) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT CAST(named_struct('a', 10) AS STRUCT).a +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +CREATE TABLE describe_col_test(c1 INT, c2 STRING, c3 DOUBLE) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`describe_col_test`, false + + +-- !query +DESCRIBE describe_col_test IDENTIFIER('c1') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DESCRIBE describe_col_test IDENTIFIER('c2') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE describe_col_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.describe_col_test + + +-- !query +CREATE TABLE struct_field_test(data STRUCT) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`struct_field_test`, false + + +-- !query +INSERT INTO struct_field_test VALUES (named_struct('field1', 42, 'field2', 'hello')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/struct_field_test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/struct_field_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`struct_field_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/struct_field_test), [data] ++- Project [named_struct(field1, col1#x.field1, field2, col1#x.field2) AS data#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT IDENTIFIER('data').IDENTIFIER('field1') FROM struct_field_test +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''field1''", + "hint" : "" + } +} + + +-- !query +SELECT IDENTIFIER('data').IDENTIFIER('field2') FROM struct_field_test +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''field2''", + "hint" : "" + } +} + + +-- !query +DROP TABLE struct_field_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.struct_field_test + + +-- !query +CREATE TABLE partition_spec_test(c1 INT, c2 STRING) USING CSV PARTITIONED BY (c2) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`partition_spec_test`, false + + +-- !query +INSERT INTO partition_spec_test PARTITION (IDENTIFIER('c2') = 'value1') VALUES (1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'IDENTIFIER'", + "hint" : "" + } +} + + +-- !query +INSERT INTO partition_spec_test PARTITION (IDENTIFIER('c2') = 'value2') VALUES (2) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'IDENTIFIER'", + "hint" : "" + } +} + + +-- !query +SELECT * FROM partition_spec_test ORDER BY c1 +-- !query analysis +Sort [c1#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.partition_spec_test + +- Relation spark_catalog.identifier_clause_test_schema.partition_spec_test[c1#x,c2#x] csv + + +-- !query +SHOW PARTITIONS partition_spec_test +-- !query analysis +ShowPartitionsCommand `spark_catalog`.`identifier_clause_test_schema`.`partition_spec_test`, [partition#x] + + +-- !query +DROP TABLE partition_spec_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.partition_spec_test + + -- !query SELECT :IDENTIFIER('param1') FROM VALUES(1) AS T(c1) -- !query analysis 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 abc6cc625b6a..82a59ad9b0ad 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 @@ -2021,22 +2021,11 @@ Project [map(mykey, 42)[mykey] AS result#x] 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')" - } ] -} +Sort [c1#x ASC NULLS FIRST], true ++- Project [c1#x] + +- SubqueryAlias t + +- 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 @@ -2182,6 +2171,233 @@ DropTable false, false +- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.unpivot_test +-- !query +CREATE TABLE unpivot_alias_test(id INT, a INT, b INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`unpivot_alias_test`, false + + +-- !query +INSERT INTO unpivot_alias_test VALUES (1, 10, 20) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/unpivot_alias_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/unpivot_alias_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`unpivot_alias_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/unpivot_alias_test), [id, a, b] ++- Project [col1#x AS id#x, col2#x AS a#x, col3#x AS b#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +SELECT * FROM unpivot_alias_test UNPIVOT (val FOR col IN (a, b)) AS IDENTIFIER('unpivoted_result') ORDER BY ALL +-- !query analysis +Sort [id#x ASC NULLS FIRST, col#x ASC NULLS FIRST, val#x ASC NULLS FIRST], true ++- Project [id#x, col#x, val#x] + +- SubqueryAlias unpivoted_result + +- Filter isnotnull(coalesce(val#x)) + +- Expand [[id#x, a, a#x], [id#x, b, b#x]], [id#x, col#x, val#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.unpivot_alias_test + +- Relation spark_catalog.identifier_clause_test_schema.unpivot_alias_test[id#x,a#x,b#x] csv + + +-- !query +DROP TABLE unpivot_alias_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.unpivot_alias_test + + +-- !query +CREATE TABLE pivot_test(product STRING, quarter STRING, revenue INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`pivot_test`, false + + +-- !query +INSERT INTO pivot_test VALUES ('A', 'Q1', 100), ('A', 'Q2', 150), ('B', 'Q1', 200), ('B', 'Q2', 250) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/pivot_test, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/pivot_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`pivot_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/pivot_test), [product, quarter, revenue] ++- Project [col1#x AS product#x, col2#x AS quarter#x, col3#x AS revenue#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +SELECT * FROM pivot_test PIVOT (SUM(revenue) FOR IDENTIFIER('quarter') IN ('Q1', 'Q2')) ORDER BY product +-- !query analysis +Sort [product#x ASC NULLS FIRST], true ++- Project [product#x, Q1#xL, Q2#xL] + +- Project [product#x, __pivot_sum(spark_catalog.identifier_clause_test_schema.pivot_test.revenue) AS `sum(spark_catalog.identifier_clause_test_schema.pivot_test.revenue)`#x[0] AS Q1#xL, __pivot_sum(spark_catalog.identifier_clause_test_schema.pivot_test.revenue) AS `sum(spark_catalog.identifier_clause_test_schema.pivot_test.revenue)`#x[1] AS Q2#xL] + +- Aggregate [product#x], [product#x, pivotfirst(quarter#x, sum(spark_catalog.identifier_clause_test_schema.pivot_test.revenue)#xL, Q1, Q2, 0, 0) AS __pivot_sum(spark_catalog.identifier_clause_test_schema.pivot_test.revenue) AS `sum(spark_catalog.identifier_clause_test_schema.pivot_test.revenue)`#x] + +- Aggregate [product#x, quarter#x], [product#x, quarter#x, sum(revenue#x) AS sum(spark_catalog.identifier_clause_test_schema.pivot_test.revenue)#xL] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.pivot_test + +- Relation spark_catalog.identifier_clause_test_schema.pivot_test[product#x,quarter#x,revenue#x] csv + + +-- !query +SELECT * FROM pivot_test PIVOT (SUM(revenue) AS IDENTIFIER('total') FOR quarter IN ('Q1' AS IDENTIFIER('first_quarter'), 'Q2' AS IDENTIFIER('second_quarter'))) ORDER BY product +-- !query analysis +Sort [product#x ASC NULLS FIRST], true ++- Project [product#x, first_quarter#xL, second_quarter#xL] + +- Project [product#x, __pivot_sum(spark_catalog.identifier_clause_test_schema.pivot_test.revenue) AS total AS `sum(spark_catalog.identifier_clause_test_schema.pivot_test.revenue) AS total`#x[0] AS first_quarter#xL, __pivot_sum(spark_catalog.identifier_clause_test_schema.pivot_test.revenue) AS total AS `sum(spark_catalog.identifier_clause_test_schema.pivot_test.revenue) AS total`#x[1] AS second_quarter#xL] + +- Aggregate [product#x], [product#x, pivotfirst(quarter#x, sum(spark_catalog.identifier_clause_test_schema.pivot_test.revenue) AS total#xL, Q1, Q2, 0, 0) AS __pivot_sum(spark_catalog.identifier_clause_test_schema.pivot_test.revenue) AS total AS `sum(spark_catalog.identifier_clause_test_schema.pivot_test.revenue) AS total`#x] + +- Aggregate [product#x, quarter#x], [product#x, quarter#x, sum(revenue#x) AS sum(spark_catalog.identifier_clause_test_schema.pivot_test.revenue) AS total#xL] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.pivot_test + +- Relation spark_catalog.identifier_clause_test_schema.pivot_test[product#x,quarter#x,revenue#x] csv + + +-- !query +DROP TABLE pivot_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.pivot_test + + +-- !query +SELECT transform(array(1, 2, 3), IDENTIFIER('x') -> x + 1) +-- !query analysis +Project [transform(array(1, 2, 3), lambdafunction((lambda x#x + 1), lambda x#x, false)) AS transform(array(1, 2, 3), lambdafunction((namedlambdavariable() + 1), namedlambdavariable()))#x] ++- OneRowRelation + + +-- !query +SELECT * FROM VALUES(1, 2) AS T(c1, c2) |> AS IDENTIFIER('pipe_alias') |> SELECT c1, c2 +-- !query analysis +Project [c1#x, c2#x] ++- SubqueryAlias pipe_alias + +- Project [c1#x, c2#x] + +- SubqueryAlias T + +- LocalRelation [c1#x, c2#x] + + +-- !query +SELECT c1, c2 FROM VALUES(1, 2) AS T(c1, c2) |> AS IDENTIFIER('my_result') |> SELECT * +-- !query analysis +Project [c1#x, c2#x] ++- SubqueryAlias my_result + +- Project [c1#x, c2#x] + +- SubqueryAlias T + +- LocalRelation [c1#x, c2#x] + + +-- !query +SELECT CAST(named_struct('field1', 1, 'field2', 'hello') AS STRUCT) +-- !query analysis +Project [cast(named_struct(field1, 1, field2, hello) as struct) AS named_struct(field1, 1, field2, hello)#x] ++- OneRowRelation + + +-- !query +SELECT CAST(named_struct('a', 10) AS STRUCT).a +-- !query analysis +Project [cast(named_struct(a, 10) as struct).a AS named_struct(a, 10).a#x] ++- OneRowRelation + + +-- !query +CREATE TABLE describe_col_test(c1 INT, c2 STRING, c3 DOUBLE) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`describe_col_test`, false + + +-- !query +DESCRIBE describe_col_test IDENTIFIER('c1') +-- !query analysis +DescribeColumnCommand `spark_catalog`.`identifier_clause_test_schema`.`describe_col_test`, [spark_catalog, identifier_clause_test_schema, describe_col_test, c1], false, [info_name#x, info_value#x] + + +-- !query +DESCRIBE describe_col_test IDENTIFIER('c2') +-- !query analysis +DescribeColumnCommand `spark_catalog`.`identifier_clause_test_schema`.`describe_col_test`, [spark_catalog, identifier_clause_test_schema, describe_col_test, c2], false, [info_name#x, info_value#x] + + +-- !query +DROP TABLE describe_col_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.describe_col_test + + +-- !query +CREATE TABLE struct_field_test(data STRUCT) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`struct_field_test`, false + + +-- !query +INSERT INTO struct_field_test VALUES (named_struct('field1', 42, 'field2', 'hello')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/struct_field_test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/struct_field_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`struct_field_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/struct_field_test), [data] ++- Project [named_struct(field1, col1#x.field1, field2, col1#x.field2) AS data#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT IDENTIFIER('data').IDENTIFIER('field1') FROM struct_field_test +-- !query analysis +Project [data#x.field1 AS data.field1#x] ++- SubqueryAlias spark_catalog.identifier_clause_test_schema.struct_field_test + +- Relation spark_catalog.identifier_clause_test_schema.struct_field_test[data#x] parquet + + +-- !query +SELECT IDENTIFIER('data').IDENTIFIER('field2') FROM struct_field_test +-- !query analysis +Project [data#x.field2 AS data.field2#x] ++- SubqueryAlias spark_catalog.identifier_clause_test_schema.struct_field_test + +- Relation spark_catalog.identifier_clause_test_schema.struct_field_test[data#x] parquet + + +-- !query +DROP TABLE struct_field_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.struct_field_test + + +-- !query +CREATE TABLE partition_spec_test(c1 INT, c2 STRING) USING CSV PARTITIONED BY (c2) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`partition_spec_test`, false + + +-- !query +INSERT INTO partition_spec_test PARTITION (IDENTIFIER('c2') = 'value1') VALUES (1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/partition_spec_test, [c2=value1], false, [c2#x], CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/partition_spec_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`partition_spec_test`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/partition_spec_test), [c1, c2] ++- Project [c1#x, cast(value1 as string) AS c2#x] + +- Project [col1#x AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO partition_spec_test PARTITION (IDENTIFIER('c2') = 'value2') VALUES (2) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/partition_spec_test, [c2=value2], false, [c2#x], CSV, [path=file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/partition_spec_test], Append, `spark_catalog`.`identifier_clause_test_schema`.`partition_spec_test`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/identifier_clause_test_schema.db/partition_spec_test), [c1, c2] ++- Project [c1#x, cast(value2 as string) AS c2#x] + +- Project [col1#x AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT * FROM partition_spec_test ORDER BY c1 +-- !query analysis +Sort [c1#x ASC NULLS FIRST], true ++- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.identifier_clause_test_schema.partition_spec_test + +- Relation spark_catalog.identifier_clause_test_schema.partition_spec_test[c1#x,c2#x] csv + + +-- !query +SHOW PARTITIONS partition_spec_test +-- !query analysis +ShowPartitionsCommand `spark_catalog`.`identifier_clause_test_schema`.`partition_spec_test`, [partition#x] + + +-- !query +DROP TABLE partition_spec_test +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.partition_spec_test + + -- !query SELECT :IDENTIFIER('param1') FROM VALUES(1) AS T(c1) -- !query analysis @@ -2294,6 +2510,138 @@ org.apache.spark.sql.AnalysisException } +-- !query +ADD IDENTIFIER('file') '/tmp/test.txt' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0035", + "messageParameters" : { + "message" : "ADD with resource type 'identifier('file')'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 38, + "fragment" : "ADD IDENTIFIER('file') '/tmp/test.txt'" + } ] +} + + +-- !query +LIST IDENTIFIER('files') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0035", + "messageParameters" : { + "message" : "LIST with resource type 'identifier('files')'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 24, + "fragment" : "LIST IDENTIFIER('files')" + } ] +} + + +-- !query +CREATE FUNCTION keyword_test_func AS 'com.example.Test' USING IDENTIFIER('jar') '/path/to.jar' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0035", + "messageParameters" : { + "message" : "CREATE FUNCTION with resource type 'identifier('jar')'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 94, + "fragment" : "CREATE FUNCTION keyword_test_func AS 'com.example.Test' USING IDENTIFIER('jar') '/path/to.jar'" + } ] +} + + +-- !query +CREATE TABLE analyze_keyword_test(c1 INT) USING CSV +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`identifier_clause_test_schema`.`analyze_keyword_test`, false + + +-- !query +ANALYZE TABLE analyze_keyword_test COMPUTE STATISTICS IDENTIFIER('noscan') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX.ANALYZE_TABLE_UNEXPECTED_NOSCAN", + "sqlState" : "42000", + "messageParameters" : { + "ctx" : "IDENTIFIER('NOSCAN')" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 74, + "fragment" : "ANALYZE TABLE analyze_keyword_test COMPUTE STATISTICS IDENTIFIER('noscan')" + } ] +} + + +-- !query +DROP TABLE IF EXISTS analyze_keyword_test +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.analyze_keyword_test + + +-- !query +CREATE TABLE transform_keyword_test(c1 INT, c2 TIMESTAMP) USING PARQUET +PARTITIONED BY (IDENTIFIER('years')(c2)) +-- !query analysis +org.apache.spark.SparkUnsupportedOperationException +{ + "errorClass" : "UNSUPPORTED_PARTITION_TRANSFORM", + "sqlState" : "0A000", + "messageParameters" : { + "transform" : "`IDENTIFIER('years')(c2)`" + } +} + + +-- !query +DROP TABLE IF EXISTS transform_keyword_test +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.transform_keyword_test + + +-- !query +CREATE TABLE bucket_keyword_test(c1 INT, c2 STRING) USING PARQUET +PARTITIONED BY (IDENTIFIER('bucket')(4, c2)) +-- !query analysis +org.apache.spark.SparkUnsupportedOperationException +{ + "errorClass" : "UNSUPPORTED_PARTITION_TRANSFORM", + "sqlState" : "0A000", + "messageParameters" : { + "transform" : "`IDENTIFIER('bucket')(4, c2)`" + } +} + + +-- !query +DROP TABLE IF EXISTS bucket_keyword_test +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clause_test_schema.bucket_keyword_test + + -- !query DROP SCHEMA identifier_clause_test_schema -- !query analysis 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 bb6c7107062d..2d6676a8f718 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 @@ -367,6 +367,62 @@ SELECT * FROM unpivot_test UNPIVOT (val FOR col IN (a AS IDENTIFIER('col_a'), b 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; +-- UNPIVOT result table alias with IDENTIFIER() +CREATE TABLE unpivot_alias_test(id INT, a INT, b INT) USING CSV; +INSERT INTO unpivot_alias_test VALUES (1, 10, 20); +SELECT * FROM unpivot_alias_test UNPIVOT (val FOR col IN (a, b)) AS IDENTIFIER('unpivoted_result') ORDER BY ALL; +DROP TABLE unpivot_alias_test; + +-- PIVOT with IDENTIFIER() for pivot column and value aliases +CREATE TABLE pivot_test(product STRING, quarter STRING, revenue INT) USING CSV; +INSERT INTO pivot_test VALUES ('A', 'Q1', 100), ('A', 'Q2', 150), ('B', 'Q1', 200), ('B', 'Q2', 250); +-- PIVOT column with IDENTIFIER() +SELECT * FROM pivot_test PIVOT (SUM(revenue) FOR IDENTIFIER('quarter') IN ('Q1', 'Q2')) ORDER BY product; +-- PIVOT value alias with IDENTIFIER() +SELECT * FROM pivot_test PIVOT (SUM(revenue) AS IDENTIFIER('total') FOR quarter IN ('Q1' AS IDENTIFIER('first_quarter'), 'Q2' AS IDENTIFIER('second_quarter'))) ORDER BY product; +DROP TABLE pivot_test; + +-- Lambda variable names with IDENTIFIER() +-- Note: Lambda variable binding with IDENTIFIER() has limitations - the variable declaration +-- and usage are resolved independently, so using IDENTIFIER() for both doesn't work as expected. +-- This test verifies the parsing works, but the semantic resolution is a known limitation. +SELECT transform(array(1, 2, 3), IDENTIFIER('x') -> x + 1); + +-- Note: INDEX tests are skipped because CreateIndex/DropIndex operations +-- are not supported on standard table types. The IDENTIFIER() syntax +-- for index names is tested at the parser level via DDLParserSuite. + +-- Pipe operator alias with IDENTIFIER() +SELECT * FROM VALUES(1, 2) AS T(c1, c2) |> AS IDENTIFIER('pipe_alias') |> SELECT c1, c2; +SELECT c1, c2 FROM VALUES(1, 2) AS T(c1, c2) |> AS IDENTIFIER('my_result') |> SELECT *; + +-- Struct field names with IDENTIFIER() in CAST +SELECT CAST(named_struct('field1', 1, 'field2', 'hello') AS STRUCT); +SELECT CAST(named_struct('a', 10) AS STRUCT).a; + +-- DESCRIBE column with IDENTIFIER() +CREATE TABLE describe_col_test(c1 INT, c2 STRING, c3 DOUBLE) USING CSV; +DESCRIBE describe_col_test IDENTIFIER('c1'); +DESCRIBE describe_col_test IDENTIFIER('c2'); +DROP TABLE describe_col_test; + +-- Qualified column name (struct field access) with IDENTIFIER() +CREATE TABLE struct_field_test(data STRUCT) USING PARQUET; +INSERT INTO struct_field_test VALUES (named_struct('field1', 42, 'field2', 'hello')); +-- Access struct using IDENTIFIER() for both struct column and field +SELECT IDENTIFIER('data').IDENTIFIER('field1') FROM struct_field_test; +SELECT IDENTIFIER('data').IDENTIFIER('field2') FROM struct_field_test; +DROP TABLE struct_field_test; + +-- Partition spec with IDENTIFIER() for partition column name +CREATE TABLE partition_spec_test(c1 INT, c2 STRING) USING CSV PARTITIONED BY (c2); +INSERT INTO partition_spec_test PARTITION (IDENTIFIER('c2') = 'value1') VALUES (1); +INSERT INTO partition_spec_test PARTITION (IDENTIFIER('c2') = 'value2') VALUES (2); +SELECT * FROM partition_spec_test ORDER BY c1; +-- Show partitions to verify +SHOW PARTITIONS partition_spec_test; +DROP TABLE partition_spec_test; + -- All the following tests fail because they are not about "true" identifiers -- This should fail - named parameters don't support IDENTIFIER() @@ -388,4 +444,23 @@ 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'); +-- Resource type is a keyword - should fail +ADD IDENTIFIER('file') '/tmp/test.txt'; +LIST IDENTIFIER('files'); +CREATE FUNCTION keyword_test_func AS 'com.example.Test' USING IDENTIFIER('jar') '/path/to.jar'; + +-- ANALYZE TABLE with NOSCAN is a keyword - should fail +CREATE TABLE analyze_keyword_test(c1 INT) USING CSV; +ANALYZE TABLE analyze_keyword_test COMPUTE STATISTICS IDENTIFIER('noscan'); +DROP TABLE IF EXISTS analyze_keyword_test; + +-- Partition transform type (bucket, years, months, days, hours) is a keyword - should fail +CREATE TABLE transform_keyword_test(c1 INT, c2 TIMESTAMP) USING PARQUET +PARTITIONED BY (IDENTIFIER('years')(c2)); +DROP TABLE IF EXISTS transform_keyword_test; + +CREATE TABLE bucket_keyword_test(c1 INT, c2 STRING) USING PARQUET +PARTITIONED BY (IDENTIFIER('bucket')(4, c2)); +DROP TABLE IF EXISTS bucket_keyword_test; + DROP SCHEMA identifier_clause_test_schema; 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 index 272d9bce8165..d18ec28dc515 100644 --- 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 @@ -2740,6 +2740,350 @@ struct<> +-- !query +CREATE TABLE unpivot_alias_test(id INT, a INT, b INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO unpivot_alias_test VALUES (1, 10, 20) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM unpivot_alias_test UNPIVOT (val FOR col IN (a, b)) AS IDENTIFIER('unpivoted_result') 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_alias_test +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE pivot_test(product STRING, quarter STRING, revenue INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO pivot_test VALUES ('A', 'Q1', 100), ('A', 'Q2', 150), ('B', 'Q1', 200), ('B', 'Q2', 250) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM pivot_test PIVOT (SUM(revenue) FOR IDENTIFIER('quarter') IN ('Q1', 'Q2')) ORDER BY product +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing 'IN'" + } +} + + +-- !query +SELECT * FROM pivot_test PIVOT (SUM(revenue) AS IDENTIFIER('total') FOR quarter IN ('Q1' AS IDENTIFIER('first_quarter'), 'Q2' AS IDENTIFIER('second_quarter'))) ORDER BY product +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing 'FOR'" + } +} + + +-- !query +DROP TABLE pivot_test +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT transform(array(1, 2, 3), IDENTIFIER('x') -> x + 1) +-- !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 VALUES(1, 2) AS T(c1, c2) |> AS IDENTIFIER('pipe_alias') |> SELECT c1, c2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT c1, c2 FROM VALUES(1, 2) AS T(c1, c2) |> AS IDENTIFIER('my_result') |> SELECT * +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT CAST(named_struct('field1', 1, 'field2', 'hello') AS STRUCT) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT CAST(named_struct('a', 10) AS STRUCT).a +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +CREATE TABLE describe_col_test(c1 INT, c2 STRING, c3 DOUBLE) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE describe_col_test IDENTIFIER('c1') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DESCRIBE describe_col_test IDENTIFIER('c2') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +DROP TABLE describe_col_test +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE struct_field_test(data STRUCT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO struct_field_test VALUES (named_struct('field1', 42, 'field2', 'hello')) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT IDENTIFIER('data').IDENTIFIER('field1') FROM struct_field_test +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''field1''", + "hint" : "" + } +} + + +-- !query +SELECT IDENTIFIER('data').IDENTIFIER('field2') FROM struct_field_test +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''field2''", + "hint" : "" + } +} + + +-- !query +DROP TABLE struct_field_test +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE partition_spec_test(c1 INT, c2 STRING) USING CSV PARTITIONED BY (c2) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO partition_spec_test PARTITION (IDENTIFIER('c2') = 'value1') VALUES (1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'IDENTIFIER'", + "hint" : "" + } +} + + +-- !query +INSERT INTO partition_spec_test PARTITION (IDENTIFIER('c2') = 'value2') VALUES (2) +-- !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 partition_spec_test ORDER BY c1 +-- !query schema +struct +-- !query output + + + +-- !query +SHOW PARTITIONS partition_spec_test +-- !query schema +struct +-- !query output + + + +-- !query +DROP TABLE partition_spec_test +-- !query schema +struct<> +-- !query output + + + -- !query SELECT :IDENTIFIER('param1') FROM VALUES(1) AS T(c1) -- !query schema 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 b398c07b14e1..6254855d221a 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 @@ -2227,24 +2227,11 @@ struct EXECUTE IMMEDIATE 'SELECT IDENTIFIER(:alias ''.c1'') FROM integration_test AS IDENTIFIER(:alias) ORDER BY ALL' USING 't' AS alias -- !query schema -struct<> +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')" - } ] -} +1 +2 +3 -- !query @@ -2396,6 +2383,247 @@ struct<> +-- !query +CREATE TABLE unpivot_alias_test(id INT, a INT, b INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO unpivot_alias_test VALUES (1, 10, 20) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM unpivot_alias_test UNPIVOT (val FOR col IN (a, b)) AS IDENTIFIER('unpivoted_result') ORDER BY ALL +-- !query schema +struct +-- !query output +1 a 10 +1 b 20 + + +-- !query +DROP TABLE unpivot_alias_test +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE pivot_test(product STRING, quarter STRING, revenue INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO pivot_test VALUES ('A', 'Q1', 100), ('A', 'Q2', 150), ('B', 'Q1', 200), ('B', 'Q2', 250) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM pivot_test PIVOT (SUM(revenue) FOR IDENTIFIER('quarter') IN ('Q1', 'Q2')) ORDER BY product +-- !query schema +struct +-- !query output +A 100 150 +B 200 250 + + +-- !query +SELECT * FROM pivot_test PIVOT (SUM(revenue) AS IDENTIFIER('total') FOR quarter IN ('Q1' AS IDENTIFIER('first_quarter'), 'Q2' AS IDENTIFIER('second_quarter'))) ORDER BY product +-- !query schema +struct +-- !query output +A 100 150 +B 200 250 + + +-- !query +DROP TABLE pivot_test +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT transform(array(1, 2, 3), IDENTIFIER('x') -> x + 1) +-- !query schema +struct> +-- !query output +[2,3,4] + + +-- !query +SELECT * FROM VALUES(1, 2) AS T(c1, c2) |> AS IDENTIFIER('pipe_alias') |> SELECT c1, c2 +-- !query schema +struct +-- !query output +1 2 + + +-- !query +SELECT c1, c2 FROM VALUES(1, 2) AS T(c1, c2) |> AS IDENTIFIER('my_result') |> SELECT * +-- !query schema +struct +-- !query output +1 2 + + +-- !query +SELECT CAST(named_struct('field1', 1, 'field2', 'hello') AS STRUCT) +-- !query schema +struct> +-- !query output +{"field1":1,"field2":"hello"} + + +-- !query +SELECT CAST(named_struct('a', 10) AS STRUCT).a +-- !query schema +struct +-- !query output +10 + + +-- !query +CREATE TABLE describe_col_test(c1 INT, c2 STRING, c3 DOUBLE) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE describe_col_test IDENTIFIER('c1') +-- !query schema +struct +-- !query output +col_name c1 +data_type int +comment NULL + + +-- !query +DESCRIBE describe_col_test IDENTIFIER('c2') +-- !query schema +struct +-- !query output +col_name c2 +data_type string +comment NULL + + +-- !query +DROP TABLE describe_col_test +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE struct_field_test(data STRUCT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO struct_field_test VALUES (named_struct('field1', 42, 'field2', 'hello')) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT IDENTIFIER('data').IDENTIFIER('field1') FROM struct_field_test +-- !query schema +struct +-- !query output +42 + + +-- !query +SELECT IDENTIFIER('data').IDENTIFIER('field2') FROM struct_field_test +-- !query schema +struct +-- !query output +hello + + +-- !query +DROP TABLE struct_field_test +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE partition_spec_test(c1 INT, c2 STRING) USING CSV PARTITIONED BY (c2) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO partition_spec_test PARTITION (IDENTIFIER('c2') = 'value1') VALUES (1) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO partition_spec_test PARTITION (IDENTIFIER('c2') = 'value2') VALUES (2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM partition_spec_test ORDER BY c1 +-- !query schema +struct +-- !query output +1 value1 +2 value2 + + +-- !query +SHOW PARTITIONS partition_spec_test +-- !query schema +struct +-- !query output +c2=value1 +c2=value2 + + +-- !query +DROP TABLE partition_spec_test +-- !query schema +struct<> +-- !query output + + + -- !query SELECT :IDENTIFIER('param1') FROM VALUES(1) AS T(c1) -- !query schema @@ -2523,6 +2751,155 @@ org.apache.spark.sql.AnalysisException } +-- !query +ADD IDENTIFIER('file') '/tmp/test.txt' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0035", + "messageParameters" : { + "message" : "ADD with resource type 'identifier('file')'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 38, + "fragment" : "ADD IDENTIFIER('file') '/tmp/test.txt'" + } ] +} + + +-- !query +LIST IDENTIFIER('files') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0035", + "messageParameters" : { + "message" : "LIST with resource type 'identifier('files')'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 24, + "fragment" : "LIST IDENTIFIER('files')" + } ] +} + + +-- !query +CREATE FUNCTION keyword_test_func AS 'com.example.Test' USING IDENTIFIER('jar') '/path/to.jar' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0035", + "messageParameters" : { + "message" : "CREATE FUNCTION with resource type 'identifier('jar')'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 94, + "fragment" : "CREATE FUNCTION keyword_test_func AS 'com.example.Test' USING IDENTIFIER('jar') '/path/to.jar'" + } ] +} + + +-- !query +CREATE TABLE analyze_keyword_test(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +ANALYZE TABLE analyze_keyword_test COMPUTE STATISTICS IDENTIFIER('noscan') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX.ANALYZE_TABLE_UNEXPECTED_NOSCAN", + "sqlState" : "42000", + "messageParameters" : { + "ctx" : "IDENTIFIER('NOSCAN')" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 74, + "fragment" : "ANALYZE TABLE analyze_keyword_test COMPUTE STATISTICS IDENTIFIER('noscan')" + } ] +} + + +-- !query +DROP TABLE IF EXISTS analyze_keyword_test +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE transform_keyword_test(c1 INT, c2 TIMESTAMP) USING PARQUET +PARTITIONED BY (IDENTIFIER('years')(c2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUnsupportedOperationException +{ + "errorClass" : "UNSUPPORTED_PARTITION_TRANSFORM", + "sqlState" : "0A000", + "messageParameters" : { + "transform" : "`IDENTIFIER('years')(c2)`" + } +} + + +-- !query +DROP TABLE IF EXISTS transform_keyword_test +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE bucket_keyword_test(c1 INT, c2 STRING) USING PARQUET +PARTITIONED BY (IDENTIFIER('bucket')(4, c2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUnsupportedOperationException +{ + "errorClass" : "UNSUPPORTED_PARTITION_TRANSFORM", + "sqlState" : "0A000", + "messageParameters" : { + "transform" : "`IDENTIFIER('bucket')(4, c2)`" + } +} + + +-- !query +DROP TABLE IF EXISTS bucket_keyword_test +-- !query schema +struct<> +-- !query output + + + -- !query DROP SCHEMA identifier_clause_test_schema -- !query schema