From c7215fb7eea23ec5b65d8bafc169c7ee8854cf17 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Thu, 1 Jun 2023 15:36:28 -0700 Subject: [PATCH 01/36] [SPARK-43922] Add named parameter support in parser for function calls --- .../apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 | 1 + .../spark/sql/catalyst/parser/SqlBaseParser.g4 | 11 ++++++++++- .../spark/sql/catalyst/parser/AstBuilder.scala | 12 +++++++++++- 3 files changed, 22 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 index 6300221b54256..9a615b3e4065a 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 @@ -442,6 +442,7 @@ CONCAT_PIPE: '||'; HAT: '^'; COLON: ':'; ARROW: '->'; +FAT_ARROW : '=>'; HENT_START: '/*+'; HENT_END: '*/'; diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 20c8df4f79aa6..57d6021b93fb2 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -854,6 +854,15 @@ expression : booleanExpression ; +namedArgumentExpression + : key = identifier FAT_ARROW expression + ; + +functionArgument + : expression + | namedArgumentExpression + ; + expressionSeq : expression (COMMA expression)* ; @@ -913,7 +922,7 @@ primaryExpression | LEFT_PAREN namedExpression (COMMA namedExpression)+ RIGHT_PAREN #rowConstructor | LEFT_PAREN query RIGHT_PAREN #subqueryExpression | IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN #identifierClause - | functionName LEFT_PAREN (setQuantifier? argument+=expression (COMMA argument+=expression)*)? RIGHT_PAREN + | functionName LEFT_PAREN (setQuantifier? argument+=functionArgument (COMMA argument+=functionArgument)*)? RIGHT_PAREN (FILTER LEFT_PAREN WHERE where=booleanExpression RIGHT_PAREN)? (nullsOption=(IGNORE | RESPECT) NULLS)? ( OVER windowSpec)? #functionCall | identifier ARROW expression #lambda 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 957c8130edec5..a40f485efd78f 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 @@ -2160,7 +2160,17 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val name = ctx.functionName.getText val isDistinct = Option(ctx.setQuantifier()).exists(_.DISTINCT != null) // Call `toSeq`, otherwise `ctx.argument.asScala.map(expression)` is `Buffer` in Scala 2.13 - val arguments = ctx.argument.asScala.map(expression).toSeq match { + val arguments = ctx.argument.asScala.map { arg => + if (arg.namedArgumentExpression != null) { + // TODO: not only could be a strictIdentifier, but also a strictNonReserved + // TODO: handle any quotes / backtick? + NamedArgumentExpression( + arg.namedArgumentExpression.key.strictIdentifier.getText, + expression(arg.namedArgumentExpression.value)) + } else { + expression(arg.expression) + } + }.toSeq match { case Seq(UnresolvedStar(None)) if name.toLowerCase(Locale.ROOT) == "count" && !isDistinct => // Transform COUNT(*) into COUNT(1). From 5e168deb03bca80fe944e0a3e52a107ebee74566 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 6 Jun 2023 11:27:46 -0700 Subject: [PATCH 02/36] Adding some compilation fixes --- .../sql/catalyst/parser/SqlBaseParser.g4 | 4 +- .../expressions/NamedArgumentExpression.scala | 54 +++++++++++++++++++ .../sql/catalyst/parser/AstBuilder.scala | 25 +++++---- 3 files changed, 72 insertions(+), 11 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 57d6021b93fb2..648d2ed048c50 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -781,7 +781,7 @@ inlineTable ; functionTable - : funcName=functionName LEFT_PAREN (expression (COMMA expression)*)? RIGHT_PAREN tableAlias + : funcName=functionName LEFT_PAREN (functionArgument (COMMA functionArgument)*)? RIGHT_PAREN tableAlias ; tableAlias @@ -855,7 +855,7 @@ expression ; namedArgumentExpression - : key = identifier FAT_ARROW expression + : key=identifier FAT_ARROW value=expression ; functionArgument diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala new file mode 100644 index 0000000000000..c6b3f99c20dc3 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.types.DataType + +/** + * An unevaluable unary expression specifically for named argument functions + * + * SQL Syntax: key => value + * SQL grammar: key=identifier FAT_ARROW value=expression + * + * NamedArgumentExpression is expected to be resolved + * and replaced in class extending [[NamedArgumentFunction]] + * + * Example usage in encode: + * SELECT encode("abc", charset => "utf-8"); + * SELECT encode(charset => "utf-8", value => "abc"); + * + * @param key The name of the function argument + * @param value The value of the function argument + */ +case class NamedArgumentExpression(key: String, value: Expression) + extends UnaryExpression with Unevaluable { + override def nullable: Boolean = value.nullable + + override def dataType: DataType = value.dataType + + override def toString: String = s"""$key => $value""" + + // NamedArgumentExpression has a single child, which is its value expression, + // so the value expression can be resolved by Analyzer rules recursively. + // For example, when the value is a built-in function expression, + // it must be resolved by [[ResolveFunctions]] + override def child: Expression = value + + override protected def withNewChildInternal(newChild: Expression): Expression = + copy(value = newChild) +} 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 a40f485efd78f..33f4de7aa8fe6 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 @@ -1527,8 +1527,17 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit if (name.length > 1) { throw QueryParsingErrors.invalidTableValuedFunctionNameError(name, ctx) } + val args = func.functionArgument.asScala.map { e => + if (e.namedArgumentExpression != null) { + val key = e.namedArgumentExpression.key.strictIdentifier + val value = e.namedArgumentExpression.value + NamedArgumentExpression(key.getText, expression(value)) + } else { + expression(e) + } + } - val tvf = UnresolvedTableValuedFunction(name, func.expression.asScala.map(expression).toSeq) + val tvf = UnresolvedTableValuedFunction(name, args) val tvfAliases = if (aliases.nonEmpty) UnresolvedTVFAliases(name, tvf, aliases) else tvf @@ -2160,15 +2169,13 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val name = ctx.functionName.getText val isDistinct = Option(ctx.setQuantifier()).exists(_.DISTINCT != null) // Call `toSeq`, otherwise `ctx.argument.asScala.map(expression)` is `Buffer` in Scala 2.13 - val arguments = ctx.argument.asScala.map { arg => - if (arg.namedArgumentExpression != null) { - // TODO: not only could be a strictIdentifier, but also a strictNonReserved - // TODO: handle any quotes / backtick? - NamedArgumentExpression( - arg.namedArgumentExpression.key.strictIdentifier.getText, - expression(arg.namedArgumentExpression.value)) + val arguments = ctx.argument.asScala.map { e => + if (e.namedArgumentExpression != null) { + val key = e.namedArgumentExpression.key.strictIdentifier + val value = e.namedArgumentExpression.value + NamedArgumentExpression(key.getText, expression(value)) } else { - expression(arg.expression) + expression(e) } }.toSeq match { case Seq(UnresolvedStar(None)) From e044d1416498c4e153829e36b8c8e27235371dab Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Mon, 12 Jun 2023 09:31:39 -0700 Subject: [PATCH 03/36] Adding tests --- .../sql/catalyst/parser/ExpressionParserSuite.scala | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 5e70402f2e7d6..098ce5b8c1b84 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -329,6 +329,18 @@ class ExpressionParserSuite extends AnalysisTest { parameters = Map("error" -> "'x'", "hint" -> ": extra input 'x'")) } + test("function expressions with named arguments") { + assertEqual("encode(value => 'abc', charset => 'utf-8')", + $"encode".function(NamedArgumentExpression("value", Literal("abc")), + NamedArgumentExpression("charset", Literal("utf-8")))) + assertEqual("encode('abc', charset => 'utf-8')", + $"encode".function(Literal("abc"), + NamedArgumentExpression("charset", Literal("utf-8")))) + assertEqual("encode(charset => 'utf-8', 'abc')", + $"encode".function( + NamedArgumentExpression("charset", Literal("utf-8")), Literal("abc"))) + } + private def lv(s: Symbol) = UnresolvedNamedLambdaVariable(Seq(s.name)) test("lambda functions") { From 6e44c55cfc8356b137a8deb0fd6b9d571d5e9d5a Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Mon, 12 Jun 2023 09:37:35 -0700 Subject: [PATCH 04/36] Adding table valued function tests --- .../sql/catalyst/parser/PlanParserSuite.scala | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 5a28ef847dcc1..77675bb27bc0c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -1412,6 +1412,31 @@ class PlanParserSuite extends AnalysisTest { assertEqual("select a, b from db.c; ;; ;", table("db", "c").select($"a", $"b")) } + test("table valued function with named arguments") { + // All named arguments + assertEqual( + "select * from my_tvf(arg1 => 'value1', arg2 => true)", + UnresolvedTableValuedFunction("my_tvf", + NamedArgumentExpression("arg1", Literal("value1")) :: + NamedArgumentExpression("arg2", Literal(true)) :: Nil).select(star())) + + // Unnamed and named arguments + assertEqual( + "select * from my_tvf(2, arg1 => 'value1', arg2 => true)", + UnresolvedTableValuedFunction("my_tvf", + Literal(2) :: + NamedArgumentExpression("arg1", Literal("value1")) :: + NamedArgumentExpression("arg2", Literal(true)) :: Nil).select(star())) + + // Mixed arguments + assertEqual( + "select * from my_tvf(arg1 => 'value1', 2, arg2 => true)", + UnresolvedTableValuedFunction("my_tvf", + NamedArgumentExpression("arg1", Literal("value1")) :: + Literal(2) :: + NamedArgumentExpression("arg2", Literal(true)) :: Nil).select(star())) + } + test("SPARK-32106: TRANSFORM plan") { // verify schema less assertEqual( From 20b9b9350b650eda7601e49249a1780b32cd3f07 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Mon, 12 Jun 2023 10:36:47 -0700 Subject: [PATCH 05/36] Adding error analysis test --- .../expressions/NamedArgumentExpression.scala | 3 - .../sql/catalyst/parser/AstBuilder.scala | 251 ++++++++---------- .../errors/QueryExecutionErrorsSuite.scala | 14 +- 3 files changed, 131 insertions(+), 137 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala index c6b3f99c20dc3..20f7f30a2806a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala @@ -25,9 +25,6 @@ import org.apache.spark.sql.types.DataType * SQL Syntax: key => value * SQL grammar: key=identifier FAT_ARROW value=expression * - * NamedArgumentExpression is expected to be resolved - * and replaced in class extending [[NamedArgumentFunction]] - * * Example usage in encode: * SELECT encode("abc", charset => "utf-8"); * SELECT encode(charset => "utf-8", value => "abc"); 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 377f83367ff68..64e0b9a09858d 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 @@ -45,7 +45,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, con import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces, TableCatalog} import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} -import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryParsingErrors} +import org.apache.spark.sql.errors.QueryParsingErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -277,10 +277,10 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit /** * Parameters used for writing query to a table: - * (table ident, tableColumnList, partitionKeys, ifPartitionNotExists, byName). + * (UnresolvedRelation, tableColumnList, partitionKeys, ifPartitionNotExists). */ - type InsertTableParams = - (IdentifierReferenceContext, Seq[String], Map[String, Option[String]], Boolean, Boolean) + type InsertTableParams = (IdentifierReferenceContext, + Seq[String], Map[String, Option[String]], Boolean) /** * Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat, provider). @@ -291,7 +291,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * Add an * {{{ * INSERT OVERWRITE TABLE tableIdentifier [partitionSpec [IF NOT EXISTS]]? [identifierList] - * INSERT INTO [TABLE] tableIdentifier [partitionSpec] ([BY NAME] | [identifierList]) + * INSERT INTO [TABLE] tableIdentifier [partitionSpec] [identifierList] * INSERT INTO [TABLE] tableIdentifier REPLACE whereClause * INSERT OVERWRITE [LOCAL] DIRECTORY STRING [rowFormat] [createFileFormat] * INSERT OVERWRITE [LOCAL] DIRECTORY [STRING] tableProvider [OPTIONS tablePropertyList] @@ -302,29 +302,25 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit ctx: InsertIntoContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) { ctx match { - // We cannot push withIdentClause() into the write command because: - // 1. `PlanWithUnresolvedIdentifier` is not a NamedRelation - // 2. Write commands do not hold the table logical plan as a child, and we need to add - // additional resolution code to resolve identifiers inside the write commands. case table: InsertIntoTableContext => - val (relationCtx, cols, partition, ifPartitionNotExists, byName) - = visitInsertIntoTable(table) - withIdentClause(relationCtx, ident => { + val (relation, cols, partition, ifPartitionNotExists) = visitInsertIntoTable(table) + // We cannot push withIdentClause() into the InsertIntoStatement because + // InsertIntoStatement() is a unary node. Changing that two binary will bypass streaming + // specific code in InsertIntoStatement resolution in the analyzer + withIdentClause(relation, ident => { InsertIntoStatement( - createUnresolvedRelation(relationCtx, ident), + createUnresolvedRelation(relation, ident), partition, cols, query, overwrite = false, - ifPartitionNotExists, - byName) + ifPartitionNotExists) }) case table: InsertOverwriteTableContext => - val (relationCtx, cols, partition, ifPartitionNotExists, _) - = visitInsertOverwriteTable(table) - withIdentClause(relationCtx, ident => { + val (relation, cols, partition, ifPartitionNotExists) = visitInsertOverwriteTable(table) + withIdentClause(relation, ident => { InsertIntoStatement( - createUnresolvedRelation(relationCtx, ident), + createUnresolvedRelation(relation, ident), partition, cols, query, @@ -361,7 +357,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit operationNotAllowed("INSERT INTO ... IF NOT EXISTS", ctx) } - (ctx.identifierReference, cols, partitionKeys, false, ctx.NAME() != null) + (ctx.identifierReference(), cols, partitionKeys, false) } /** @@ -379,7 +375,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit dynamicPartitionKeys.keys.mkString(", "), ctx) } - (ctx.identifierReference, cols, partitionKeys, ctx.EXISTS() != null, false) + (ctx.identifierReference(), cols, partitionKeys, ctx.EXISTS() != null) } /** @@ -413,7 +409,9 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit override def visitDeleteFromTable( ctx: DeleteFromTableContext): LogicalPlan = withOrigin(ctx) { - val table = createUnresolvedRelation(ctx.identifierReference) + val table = withIdentClause( + ctx.identifierReference, + createUnresolvedRelation(ctx.identifierReference, _)) val tableAlias = getTableAliasWithoutColumnAlias(ctx.tableAlias(), "DELETE") val aliasedTable = tableAlias.map(SubqueryAlias(_, table)).getOrElse(table) val predicate = if (ctx.whereClause() != null) { @@ -425,7 +423,9 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } override def visitUpdateTable(ctx: UpdateTableContext): LogicalPlan = withOrigin(ctx) { - val table = createUnresolvedRelation(ctx.identifierReference) + val table = withIdentClause( + ctx.identifierReference, + createUnresolvedRelation(ctx.identifierReference, _)) val tableAlias = getTableAliasWithoutColumnAlias(ctx.tableAlias(), "UPDATE") val aliasedTable = tableAlias.map(SubqueryAlias(_, table)).getOrElse(table) val assignments = withAssignments(ctx.setClause().assignmentList()) @@ -447,12 +447,12 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } override def visitMergeIntoTable(ctx: MergeIntoTableContext): LogicalPlan = withOrigin(ctx) { - val targetTable = createUnresolvedRelation(ctx.target) + val targetTable = withIdentClause(ctx.target, createUnresolvedRelation(ctx.target, _ )) val targetTableAlias = getTableAliasWithoutColumnAlias(ctx.targetAlias, "MERGE") val aliasedTarget = targetTableAlias.map(SubqueryAlias(_, targetTable)).getOrElse(targetTable) val sourceTableOrQuery = if (ctx.source != null) { - createUnresolvedRelation(ctx.source) + withIdentClause(ctx.source, createUnresolvedRelation(ctx.source, _)) } else if (ctx.sourceQuery != null) { visitQuery(ctx.sourceQuery) } else { @@ -1488,14 +1488,14 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * }}} */ override def visitTable(ctx: TableContext): LogicalPlan = withOrigin(ctx) { - createUnresolvedRelation(ctx.identifierReference) + withIdentClause(ctx.identifierReference, UnresolvedRelation(_)) } /** * Create an aliased table reference. This is typically used in FROM clauses. */ override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) { - val relation = createUnresolvedRelation(ctx.identifierReference) + val relation = withIdentClause(ctx.identifierReference, UnresolvedRelation(_)) val table = mayApplyAliasPlan( ctx.tableAlias, relation.optionalMap(ctx.temporalClause)(withTimeTravel)) table.optionalMap(ctx.sample)(withSample) @@ -1536,31 +1536,26 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } else { Seq.empty } - - withIdentClause( - func.functionName.expression, - () => getFunctionMultiparts(func.functionName), - name => { - if (name.length > 1) { - throw QueryParsingErrors.invalidTableValuedFunctionNameError(name, ctx) - } - val args = func.functionArgument.asScala.map { e => - if (e.namedArgumentExpression != null) { - val key = e.namedArgumentExpression.key.strictIdentifier - val value = e.namedArgumentExpression.value - NamedArgumentExpression(key.getText, expression(value)) - } else { - expression(e) - } + withFuncIdentClause(func.functionName, name => { + if (name.length > 1) { + throw QueryParsingErrors.invalidTableValuedFunctionNameError(name, ctx) + } + val args = func.functionArgument.asScala.map { e => + if (e.namedArgumentExpression != null) { + val key = e.namedArgumentExpression.key.strictIdentifier + val value = e.namedArgumentExpression.value + NamedArgumentExpression(key.getText, expression(value)) + } else { + expression(e) } + } - val tvf = UnresolvedTableValuedFunction(name, args) + val tvf = UnresolvedTableValuedFunction(name, args) - val tvfAliases = if (aliases.nonEmpty) UnresolvedTVFAliases(name, tvf, aliases) else tvf + val tvfAliases = if (aliases.nonEmpty) UnresolvedTVFAliases(name, tvf, aliases) else tvf - tvfAliases.optionalMap(func.tableAlias.strictIdentifier)(aliasPlan) - } - ) + tvfAliases.optionalMap(func.tableAlias.strictIdentifier)(aliasPlan) + }) } /** @@ -2171,10 +2166,12 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } /** - * Create an expression for the IDENTIFIER() clause. + * Create a IDENTIFIER() clause. */ override def visitIdentifierClause(ctx: IdentifierClauseContext): Expression = withOrigin(ctx) { - ExpressionWithUnresolvedIdentifier(expression(ctx.expression), UnresolvedAttribute(_)) + // Create the function call. + val expr: Expression = expression(ctx.expression()) + UnresolvedAttributeIdentifierClause(expr) } /** @@ -2204,18 +2201,19 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val filter = Option(ctx.where).map(expression(_)) val ignoreNulls = Option(ctx.nullsOption).map(_.getType == SqlBaseParser.IGNORE).getOrElse(false) - val funcCtx = ctx.functionName - val func = withIdentClause(funcCtx.expression, () => getFunctionMultiparts(funcCtx), ident => { - UnresolvedFunction(ident, arguments, isDistinct, filter, ignoreNulls) - }) + val function = Option(ctx.functionName.expression()).map(p => + UnresolvedFunctionIdentifierClause(expression(p), + arguments, isDistinct, filter, ignoreNulls)).getOrElse( + UnresolvedFunction(getFunctionMultiparts(ctx.functionName), + arguments, isDistinct, filter, ignoreNulls)) // Check if the function is evaluated in a windowed context. ctx.windowSpec match { case spec: WindowRefContext => - UnresolvedWindowExpression(func, visitWindowRef(spec)) + UnresolvedWindowExpression(function, visitWindowRef(spec)) case spec: WindowDefContext => - WindowExpression(func, visitWindowDef(spec)) - case _ => func + WindowExpression(function, visitWindowDef(spec)) + case _ => function } } @@ -2719,14 +2717,6 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } } - /** - * Create an [[UnresolvedRelation]] from an identifier reference. - */ - private def createUnresolvedRelation( - ctx: IdentifierReferenceContext): LogicalPlan = withOrigin(ctx) { - withIdentClause(ctx, UnresolvedRelation(_)) - } - /** * Create an [[UnresolvedRelation]] from a multi-part identifier. */ @@ -2770,7 +2760,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit ctx: ParserRuleContext, ident: Seq[String], commandName: String, - allowTempView: Boolean): UnresolvedTableOrView = withOrigin(ctx) { + allowTempView: Boolean ): UnresolvedTableOrView = withOrigin(ctx) { UnresolvedTableOrView(ident, commandName, allowTempView) } @@ -3360,20 +3350,6 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } } - /** - * Parse a key-value map from an [[ExpressionPropertyListContext]], assuming all values are - * specified. - */ - override def visitExpressionPropertyList( - ctx: ExpressionPropertyListContext): OptionsListExpressions = { - val options = ctx.expressionProperty.asScala.map { property => - val key: String = visitPropertyKey(property.key) - val value: Expression = Option(property.value).map(expression).getOrElse(null) - key -> value - }.toSeq - OptionsListExpressions(options) - } - override def visitStringLit(ctx: StringLitContext): Token = { if (ctx != null) { if (ctx.STRING_LITERAL != null) { @@ -3408,7 +3384,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit */ type TableClauses = ( Seq[Transform], Seq[StructField], Option[BucketSpec], Map[String, String], - OptionsListExpressions, Option[String], Option[String], Option[SerdeInfo]) + Map[String, String], Option[String], Option[String], Option[SerdeInfo]) /** * Validate a create table statement and return the [[TableIdentifier]]. @@ -3668,8 +3644,8 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit ctx.EXTENDED != null) } - def cleanTableProperties[ValueType]( - ctx: ParserRuleContext, properties: Map[String, ValueType]): Map[String, ValueType] = { + def cleanTableProperties( + ctx: ParserRuleContext, properties: Map[String, String]): Map[String, String] = { import TableCatalog._ val legacyOn = conf.getConf(SQLConf.LEGACY_PROPERTY_NON_RESERVED) properties.filter { @@ -3703,26 +3679,18 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit def cleanTableOptions( ctx: ParserRuleContext, - options: OptionsListExpressions, - location: Option[String]): (OptionsListExpressions, Option[String]) = { + options: Map[String, String], + location: Option[String]): (Map[String, String], Option[String]) = { var path = location - val filtered = cleanTableProperties(ctx, options.options.toMap).filter { - case (key, value) if key.equalsIgnoreCase("path") => - val newValue: String = - if (value == null) { - "" - } else value match { - case Literal(_, _: StringType) => value.toString - case _ => throw QueryCompilationErrors.optionMustBeLiteralString(key) - } - if (path.nonEmpty) { - throw QueryParsingErrors.duplicatedTablePathsFoundError(path.get, newValue, ctx) - } - path = Some(newValue) + val filtered = cleanTableProperties(ctx, options).filter { + case (k, v) if k.equalsIgnoreCase("path") && path.nonEmpty => + throw QueryParsingErrors.duplicatedTablePathsFoundError(path.get, v, ctx) + case (k, v) if k.equalsIgnoreCase("path") => + path = Some(v) false case _ => true } - (OptionsListExpressions(filtered.toSeq), path) + (filtered, path) } /** @@ -3880,8 +3848,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val bucketSpec = ctx.bucketSpec().asScala.headOption.map(visitBucketSpec) val properties = Option(ctx.tableProps).map(visitPropertyKeyValues).getOrElse(Map.empty) val cleanedProperties = cleanTableProperties(ctx, properties) - val options = Option(ctx.options).map(visitExpressionPropertyList) - .getOrElse(OptionsListExpressions(Seq.empty)) + val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) val location = visitLocationSpecList(ctx.locationSpec()) val (cleanedOptions, newLocation) = cleanTableOptions(ctx, options, location) val comment = visitCommentSpecList(ctx.commentSpec()) @@ -3961,8 +3928,8 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val columns = Option(ctx.createOrReplaceTableColTypeList()) .map(visitCreateOrReplaceTableColTypeList).getOrElse(Nil) val provider = Option(ctx.tableProvider).map(_.multipartIdentifier.getText) - val (partTransforms, partCols, bucketSpec, properties, options, location, - comment, serdeInfo) = visitCreateTableClauses(ctx.createTableClauses()) + val (partTransforms, partCols, bucketSpec, properties, options, location, comment, serdeInfo) = + visitCreateTableClauses(ctx.createTableClauses()) if (provider.isDefined && serdeInfo.isDefined) { operationNotAllowed(s"CREATE TABLE ... USING ... ${serdeInfo.get.describe}", ctx) @@ -3976,7 +3943,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val partitioning = partitionExpressions(partTransforms, partCols, ctx) ++ bucketSpec.map(_.asTransform) - val tableSpec = UnresolvedTableSpec(properties, provider, location, comment, + val tableSpec = TableSpec(properties, provider, options, location, comment, serdeInfo, external) Option(ctx.query).map(plan) match { @@ -3993,15 +3960,14 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit case Some(query) => CreateTableAsSelect(withIdentClause(identifierContext, UnresolvedIdentifier(_)), - partitioning, query, tableSpec, Map.empty, ifNotExists, optionsListExpressions = options) + partitioning, query, tableSpec, Map.empty, ifNotExists) case _ => // Note: table schema includes both the table columns list and the partition columns // with data type. val schema = StructType(columns ++ partCols) CreateTable(withIdentClause(identifierContext, UnresolvedIdentifier(_)), - schema, partitioning, tableSpec, ignoreIfExists = ifNotExists, - optionsListExpressions = options) + schema, partitioning, tableSpec, ignoreIfExists = ifNotExists) } } @@ -4046,8 +4012,8 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val partitioning = partitionExpressions(partTransforms, partCols, ctx) ++ bucketSpec.map(_.asTransform) - val tableSpec = UnresolvedTableSpec(properties, provider, location, comment, - serdeInfo, external = false) + val tableSpec = TableSpec(properties, provider, options, location, comment, + serdeInfo, false) Option(ctx.query).map(plan) match { case Some(_) if columns.nonEmpty => @@ -4064,8 +4030,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit case Some(query) => ReplaceTableAsSelect( withIdentClause(ctx.replaceTableHeader.identifierReference(), UnresolvedIdentifier(_)), - partitioning, query, tableSpec, writeOptions = Map.empty, orCreate = orCreate, - optionsListExpressions = options) + partitioning, query, tableSpec, writeOptions = Map.empty, orCreate = orCreate) case _ => // Note: table schema includes both the table columns list and the partition columns @@ -4073,7 +4038,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val schema = StructType(columns ++ partCols) ReplaceTable( withIdentClause(ctx.replaceTableHeader.identifierReference(), UnresolvedIdentifier(_)), - schema, partitioning, tableSpec, orCreate = orCreate, optionsListExpressions = options) + schema, partitioning, tableSpec, orCreate = orCreate) } } @@ -4108,12 +4073,15 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * Create a [[ShowTables]] command. */ override def visitShowTables(ctx: ShowTablesContext): LogicalPlan = withOrigin(ctx) { - val ns = if (ctx.identifierReference() != null) { - withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) + if (ctx.identifierReference() != null) { + ShowTables( + withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)), + Option(ctx.pattern).map(x => string(visitStringLit(x)))) } else { - UnresolvedNamespace(Seq.empty[String]) + ShowTables( + UnresolvedNamespace(Seq.empty[String]), + Option(ctx.pattern).map(x => string(visitStringLit(x)))) } - ShowTables(ns, Option(ctx.pattern).map(x => string(visitStringLit(x)))) } /** @@ -4124,24 +4092,32 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val partitionKeys = Option(ctx.partitionSpec).map { specCtx => UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(specCtx), None) } - val ns = if (ctx.identifierReference() != null) { - withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) + if (ctx.identifierReference() != null) { + ShowTableExtended( + withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)), + string(visitStringLit(ctx.pattern)), + partitionKeys) } else { - UnresolvedNamespace(Seq.empty[String]) + ShowTableExtended( + UnresolvedNamespace(Seq.empty[String]), + string(visitStringLit(ctx.pattern)), + partitionKeys) } - ShowTableExtended(ns, string(visitStringLit(ctx.pattern)), partitionKeys) } /** * Create a [[ShowViews]] command. */ override def visitShowViews(ctx: ShowViewsContext): LogicalPlan = withOrigin(ctx) { - val ns = if (ctx.identifierReference() != null) { - withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) + if (ctx.identifierReference() != null) { + ShowViews( + withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)), + Option(ctx.pattern).map(x => string(visitStringLit(x)))) } else { - UnresolvedNamespace(Seq.empty[String]) + ShowViews( + UnresolvedNamespace(Seq.empty[String]), + Option(ctx.pattern).map(x => string(visitStringLit(x)))) } - ShowViews(ns, Option(ctx.pattern).map(x => string(visitStringLit(x)))) } override def visitColPosition(ctx: ColPositionContext): ColumnPosition = { @@ -4602,12 +4578,15 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit ctx.identifier.getText.toLowerCase(Locale.ROOT) != "noscan") { throw QueryParsingErrors.computeStatisticsNotExpectedError(ctx.identifier()) } - val ns = if (ctx.identifierReference() != null) { - withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) + if (ctx.identifierReference() != null) { + AnalyzeTables( + withIdentClause( + ctx.identifierReference, + UnresolvedNamespace(_)), + noScan = ctx.identifier != null) } else { - UnresolvedNamespace(Seq.empty[String]) + AnalyzeTables(UnresolvedNamespace(Seq.empty[String]), noScan = ctx.identifier != null) } - AnalyzeTables(ns, noScan = ctx.identifier != null) } /** @@ -4680,17 +4659,19 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val query = Option(ctx.query).map(plan) withIdentClause(ctx.identifierReference, ident => { - if (query.isDefined && ident.length > 1) { - val catalogAndNamespace = ident.init + val relation = createUnresolvedRelation(ctx.identifierReference, ident) + val tableName = relation.multipartIdentifier + if (query.isDefined && tableName.length > 1) { + val catalogAndNamespace = tableName.init throw QueryParsingErrors.addCatalogInCacheTableAsSelectNotAllowedError( catalogAndNamespace.quoted, ctx) } val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) val isLazy = ctx.LAZY != null if (query.isDefined) { - CacheTableAsSelect(ident.head, query.get, source(ctx.query()), isLazy, options) + CacheTableAsSelect(tableName.head, query.get, source(ctx.query()), isLazy, options) } else { - CacheTable(createUnresolvedRelation(ctx.identifierReference, ident), ident, isLazy, options) + CacheTable(relation, tableName, isLazy, options) } }) } @@ -4699,7 +4680,11 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * Create an [[UncacheTable]] logical plan. */ override def visitUncacheTable(ctx: UncacheTableContext): LogicalPlan = withOrigin(ctx) { - UncacheTable(createUnresolvedRelation(ctx.identifierReference), ctx.EXISTS != null) + UncacheTable( + withIdentClause( + ctx.identifierReference, + createUnresolvedRelation(ctx.identifierReference, _)), + ctx.EXISTS != null) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index 069fce237f2b4..986e7534a1eba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark._ import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{Parameter, UnresolvedGenerator} -import org.apache.spark.sql.catalyst.expressions.{Grouping, Literal, RowNumber} +import org.apache.spark.sql.catalyst.expressions.{Grouping, Literal, NamedArgumentExpression, RowNumber} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.expressions.objects.InitializeJavaBean @@ -837,6 +837,18 @@ class QueryExecutionErrorsSuite sqlState = "XX000") } + test("INTERNAL_ERROR: Calling eval on Unevaluable NamedArgumentExpression") { + val e = intercept[SparkException] { + NamedArgumentExpression("arg0", Literal("value0")).eval() + } + checkError( + exception = e, + errorClass = "INTERNAL_ERROR", + parameters = Map("message" -> "Cannot evaluate expression: arg0 => value0"), + sqlState = "XX000" + ) + } + test("INTERNAL_ERROR: Calling doGenCode on unresolved") { val e = intercept[SparkException] { val ctx = new CodegenContext From 12c814b1263d7a8cf9b05195f6a0bb6e40a0f0ed Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Mon, 12 Jun 2023 10:48:18 -0700 Subject: [PATCH 06/36] Addressing some comments --- .../sql/catalyst/parser/AstBuilder.scala | 255 ++++---- .../errors/QueryExecutionErrorsSuite.scala | 2 +- weird_changes.diff | 586 ++++++++++++++++++ 3 files changed, 722 insertions(+), 121 deletions(-) create mode 100644 weird_changes.diff 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 64e0b9a09858d..d3ec4c1d9dd5d 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 @@ -45,7 +45,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, con import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces, TableCatalog} import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} -import org.apache.spark.sql.errors.QueryParsingErrors +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryParsingErrors} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -277,10 +277,10 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit /** * Parameters used for writing query to a table: - * (UnresolvedRelation, tableColumnList, partitionKeys, ifPartitionNotExists). + * (table ident, tableColumnList, partitionKeys, ifPartitionNotExists, byName). */ - type InsertTableParams = (IdentifierReferenceContext, - Seq[String], Map[String, Option[String]], Boolean) + type InsertTableParams = + (IdentifierReferenceContext, Seq[String], Map[String, Option[String]], Boolean, Boolean) /** * Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat, provider). @@ -291,7 +291,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * Add an * {{{ * INSERT OVERWRITE TABLE tableIdentifier [partitionSpec [IF NOT EXISTS]]? [identifierList] - * INSERT INTO [TABLE] tableIdentifier [partitionSpec] [identifierList] + * INSERT INTO [TABLE] tableIdentifier [partitionSpec] ([BY NAME] | [identifierList]) * INSERT INTO [TABLE] tableIdentifier REPLACE whereClause * INSERT OVERWRITE [LOCAL] DIRECTORY STRING [rowFormat] [createFileFormat] * INSERT OVERWRITE [LOCAL] DIRECTORY [STRING] tableProvider [OPTIONS tablePropertyList] @@ -302,25 +302,29 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit ctx: InsertIntoContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) { ctx match { + // We cannot push withIdentClause() into the write command because: + // 1. `PlanWithUnresolvedIdentifier` is not a NamedRelation + // 2. Write commands do not hold the table logical plan as a child, and we need to add + // additional resolution code to resolve identifiers inside the write commands. case table: InsertIntoTableContext => - val (relation, cols, partition, ifPartitionNotExists) = visitInsertIntoTable(table) - // We cannot push withIdentClause() into the InsertIntoStatement because - // InsertIntoStatement() is a unary node. Changing that two binary will bypass streaming - // specific code in InsertIntoStatement resolution in the analyzer - withIdentClause(relation, ident => { + val (relationCtx, cols, partition, ifPartitionNotExists, byName) + = visitInsertIntoTable(table) + withIdentClause(relationCtx, ident => { InsertIntoStatement( - createUnresolvedRelation(relation, ident), + createUnresolvedRelation(relationCtx, ident), partition, cols, query, overwrite = false, - ifPartitionNotExists) + ifPartitionNotExists, + byName) }) case table: InsertOverwriteTableContext => - val (relation, cols, partition, ifPartitionNotExists) = visitInsertOverwriteTable(table) - withIdentClause(relation, ident => { + val (relationCtx, cols, partition, ifPartitionNotExists, _) + = visitInsertOverwriteTable(table) + withIdentClause(relationCtx, ident => { InsertIntoStatement( - createUnresolvedRelation(relation, ident), + createUnresolvedRelation(relationCtx, ident), partition, cols, query, @@ -357,7 +361,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit operationNotAllowed("INSERT INTO ... IF NOT EXISTS", ctx) } - (ctx.identifierReference(), cols, partitionKeys, false) + (ctx.identifierReference, cols, partitionKeys, false, ctx.NAME() != null) } /** @@ -375,7 +379,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit dynamicPartitionKeys.keys.mkString(", "), ctx) } - (ctx.identifierReference(), cols, partitionKeys, ctx.EXISTS() != null) + (ctx.identifierReference, cols, partitionKeys, ctx.EXISTS() != null, false) } /** @@ -409,9 +413,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit override def visitDeleteFromTable( ctx: DeleteFromTableContext): LogicalPlan = withOrigin(ctx) { - val table = withIdentClause( - ctx.identifierReference, - createUnresolvedRelation(ctx.identifierReference, _)) + val table = createUnresolvedRelation(ctx.identifierReference) val tableAlias = getTableAliasWithoutColumnAlias(ctx.tableAlias(), "DELETE") val aliasedTable = tableAlias.map(SubqueryAlias(_, table)).getOrElse(table) val predicate = if (ctx.whereClause() != null) { @@ -423,9 +425,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } override def visitUpdateTable(ctx: UpdateTableContext): LogicalPlan = withOrigin(ctx) { - val table = withIdentClause( - ctx.identifierReference, - createUnresolvedRelation(ctx.identifierReference, _)) + val table = createUnresolvedRelation(ctx.identifierReference) val tableAlias = getTableAliasWithoutColumnAlias(ctx.tableAlias(), "UPDATE") val aliasedTable = tableAlias.map(SubqueryAlias(_, table)).getOrElse(table) val assignments = withAssignments(ctx.setClause().assignmentList()) @@ -447,12 +447,12 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } override def visitMergeIntoTable(ctx: MergeIntoTableContext): LogicalPlan = withOrigin(ctx) { - val targetTable = withIdentClause(ctx.target, createUnresolvedRelation(ctx.target, _ )) + val targetTable = createUnresolvedRelation(ctx.target) val targetTableAlias = getTableAliasWithoutColumnAlias(ctx.targetAlias, "MERGE") val aliasedTarget = targetTableAlias.map(SubqueryAlias(_, targetTable)).getOrElse(targetTable) val sourceTableOrQuery = if (ctx.source != null) { - withIdentClause(ctx.source, createUnresolvedRelation(ctx.source, _)) + createUnresolvedRelation(ctx.source) } else if (ctx.sourceQuery != null) { visitQuery(ctx.sourceQuery) } else { @@ -1488,14 +1488,14 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * }}} */ override def visitTable(ctx: TableContext): LogicalPlan = withOrigin(ctx) { - withIdentClause(ctx.identifierReference, UnresolvedRelation(_)) + createUnresolvedRelation(ctx.identifierReference) } /** * Create an aliased table reference. This is typically used in FROM clauses. */ override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) { - val relation = withIdentClause(ctx.identifierReference, UnresolvedRelation(_)) + val relation = createUnresolvedRelation(ctx.identifierReference) val table = mayApplyAliasPlan( ctx.tableAlias, relation.optionalMap(ctx.temporalClause)(withTimeTravel)) table.optionalMap(ctx.sample)(withSample) @@ -1536,26 +1536,31 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } else { Seq.empty } - withFuncIdentClause(func.functionName, name => { - if (name.length > 1) { - throw QueryParsingErrors.invalidTableValuedFunctionNameError(name, ctx) - } - val args = func.functionArgument.asScala.map { e => - if (e.namedArgumentExpression != null) { - val key = e.namedArgumentExpression.key.strictIdentifier - val value = e.namedArgumentExpression.value - NamedArgumentExpression(key.getText, expression(value)) - } else { - expression(e) + + withIdentClause( + func.functionName.expression, + () => getFunctionMultiparts(func.functionName), + name => { + if (name.length > 1) { + throw QueryParsingErrors.invalidTableValuedFunctionNameError(name, ctx) + } + val args = func.functionArgument.asScala.map { e => + if (e.namedArgumentExpression != null) { + val key = e.namedArgumentExpression.key.getText + val value = e.namedArgumentExpression.value + NamedArgumentExpression(key, expression(value)) + } else { + expression(e) + } } - } - val tvf = UnresolvedTableValuedFunction(name, args) + val tvf = UnresolvedTableValuedFunction(name, args) - val tvfAliases = if (aliases.nonEmpty) UnresolvedTVFAliases(name, tvf, aliases) else tvf + val tvfAliases = if (aliases.nonEmpty) UnresolvedTVFAliases(name, tvf, aliases) else tvf - tvfAliases.optionalMap(func.tableAlias.strictIdentifier)(aliasPlan) - }) + tvfAliases.optionalMap(func.tableAlias.strictIdentifier)(aliasPlan) + } + ) } /** @@ -2166,12 +2171,10 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } /** - * Create a IDENTIFIER() clause. + * Create an expression for the IDENTIFIER() clause. */ override def visitIdentifierClause(ctx: IdentifierClauseContext): Expression = withOrigin(ctx) { - // Create the function call. - val expr: Expression = expression(ctx.expression()) - UnresolvedAttributeIdentifierClause(expr) + ExpressionWithUnresolvedIdentifier(expression(ctx.expression), UnresolvedAttribute(_)) } /** @@ -2184,9 +2187,9 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit // Call `toSeq`, otherwise `ctx.argument.asScala.map(expression)` is `Buffer` in Scala 2.13 val arguments = ctx.argument.asScala.map { e => if (e.namedArgumentExpression != null) { - val key = e.namedArgumentExpression.key.strictIdentifier + val key = e.namedArgumentExpression.key.getText val value = e.namedArgumentExpression.value - NamedArgumentExpression(key.getText, expression(value)) + NamedArgumentExpression(key, expression(value)) } else { expression(e) } @@ -2201,19 +2204,18 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val filter = Option(ctx.where).map(expression(_)) val ignoreNulls = Option(ctx.nullsOption).map(_.getType == SqlBaseParser.IGNORE).getOrElse(false) - val function = Option(ctx.functionName.expression()).map(p => - UnresolvedFunctionIdentifierClause(expression(p), - arguments, isDistinct, filter, ignoreNulls)).getOrElse( - UnresolvedFunction(getFunctionMultiparts(ctx.functionName), - arguments, isDistinct, filter, ignoreNulls)) + val funcCtx = ctx.functionName + val func = withIdentClause(funcCtx.expression, () => getFunctionMultiparts(funcCtx), ident => { + UnresolvedFunction(ident, arguments, isDistinct, filter, ignoreNulls) + }) // Check if the function is evaluated in a windowed context. ctx.windowSpec match { case spec: WindowRefContext => - UnresolvedWindowExpression(function, visitWindowRef(spec)) + UnresolvedWindowExpression(func, visitWindowRef(spec)) case spec: WindowDefContext => - WindowExpression(function, visitWindowDef(spec)) - case _ => function + WindowExpression(func, visitWindowDef(spec)) + case _ => func } } @@ -2717,6 +2719,14 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } } + /** + * Create an [[UnresolvedRelation]] from an identifier reference. + */ + private def createUnresolvedRelation( + ctx: IdentifierReferenceContext): LogicalPlan = withOrigin(ctx) { + withIdentClause(ctx, UnresolvedRelation(_)) + } + /** * Create an [[UnresolvedRelation]] from a multi-part identifier. */ @@ -2760,7 +2770,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit ctx: ParserRuleContext, ident: Seq[String], commandName: String, - allowTempView: Boolean ): UnresolvedTableOrView = withOrigin(ctx) { + allowTempView: Boolean): UnresolvedTableOrView = withOrigin(ctx) { UnresolvedTableOrView(ident, commandName, allowTempView) } @@ -3350,6 +3360,20 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } } + /** + * Parse a key-value map from an [[ExpressionPropertyListContext]], assuming all values are + * specified. + */ + override def visitExpressionPropertyList( + ctx: ExpressionPropertyListContext): OptionsListExpressions = { + val options = ctx.expressionProperty.asScala.map { property => + val key: String = visitPropertyKey(property.key) + val value: Expression = Option(property.value).map(expression).getOrElse(null) + key -> value + }.toSeq + OptionsListExpressions(options) + } + override def visitStringLit(ctx: StringLitContext): Token = { if (ctx != null) { if (ctx.STRING_LITERAL != null) { @@ -3384,7 +3408,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit */ type TableClauses = ( Seq[Transform], Seq[StructField], Option[BucketSpec], Map[String, String], - Map[String, String], Option[String], Option[String], Option[SerdeInfo]) + OptionsListExpressions, Option[String], Option[String], Option[SerdeInfo]) /** * Validate a create table statement and return the [[TableIdentifier]]. @@ -3644,8 +3668,8 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit ctx.EXTENDED != null) } - def cleanTableProperties( - ctx: ParserRuleContext, properties: Map[String, String]): Map[String, String] = { + def cleanTableProperties[ValueType]( + ctx: ParserRuleContext, properties: Map[String, ValueType]): Map[String, ValueType] = { import TableCatalog._ val legacyOn = conf.getConf(SQLConf.LEGACY_PROPERTY_NON_RESERVED) properties.filter { @@ -3679,18 +3703,26 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit def cleanTableOptions( ctx: ParserRuleContext, - options: Map[String, String], - location: Option[String]): (Map[String, String], Option[String]) = { + options: OptionsListExpressions, + location: Option[String]): (OptionsListExpressions, Option[String]) = { var path = location - val filtered = cleanTableProperties(ctx, options).filter { - case (k, v) if k.equalsIgnoreCase("path") && path.nonEmpty => - throw QueryParsingErrors.duplicatedTablePathsFoundError(path.get, v, ctx) - case (k, v) if k.equalsIgnoreCase("path") => - path = Some(v) + val filtered = cleanTableProperties(ctx, options.options.toMap).filter { + case (key, value) if key.equalsIgnoreCase("path") => + val newValue: String = + if (value == null) { + "" + } else value match { + case Literal(_, _: StringType) => value.toString + case _ => throw QueryCompilationErrors.optionMustBeLiteralString(key) + } + if (path.nonEmpty) { + throw QueryParsingErrors.duplicatedTablePathsFoundError(path.get, newValue, ctx) + } + path = Some(newValue) false case _ => true } - (filtered, path) + (OptionsListExpressions(filtered.toSeq), path) } /** @@ -3848,7 +3880,8 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val bucketSpec = ctx.bucketSpec().asScala.headOption.map(visitBucketSpec) val properties = Option(ctx.tableProps).map(visitPropertyKeyValues).getOrElse(Map.empty) val cleanedProperties = cleanTableProperties(ctx, properties) - val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) + val options = Option(ctx.options).map(visitExpressionPropertyList) + .getOrElse(OptionsListExpressions(Seq.empty)) val location = visitLocationSpecList(ctx.locationSpec()) val (cleanedOptions, newLocation) = cleanTableOptions(ctx, options, location) val comment = visitCommentSpecList(ctx.commentSpec()) @@ -3928,8 +3961,8 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val columns = Option(ctx.createOrReplaceTableColTypeList()) .map(visitCreateOrReplaceTableColTypeList).getOrElse(Nil) val provider = Option(ctx.tableProvider).map(_.multipartIdentifier.getText) - val (partTransforms, partCols, bucketSpec, properties, options, location, comment, serdeInfo) = - visitCreateTableClauses(ctx.createTableClauses()) + val (partTransforms, partCols, bucketSpec, properties, options, location, + comment, serdeInfo) = visitCreateTableClauses(ctx.createTableClauses()) if (provider.isDefined && serdeInfo.isDefined) { operationNotAllowed(s"CREATE TABLE ... USING ... ${serdeInfo.get.describe}", ctx) @@ -3943,7 +3976,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val partitioning = partitionExpressions(partTransforms, partCols, ctx) ++ bucketSpec.map(_.asTransform) - val tableSpec = TableSpec(properties, provider, options, location, comment, + val tableSpec = UnresolvedTableSpec(properties, provider, location, comment, serdeInfo, external) Option(ctx.query).map(plan) match { @@ -3960,14 +3993,15 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit case Some(query) => CreateTableAsSelect(withIdentClause(identifierContext, UnresolvedIdentifier(_)), - partitioning, query, tableSpec, Map.empty, ifNotExists) + partitioning, query, tableSpec, Map.empty, ifNotExists, optionsListExpressions = options) case _ => // Note: table schema includes both the table columns list and the partition columns // with data type. val schema = StructType(columns ++ partCols) CreateTable(withIdentClause(identifierContext, UnresolvedIdentifier(_)), - schema, partitioning, tableSpec, ignoreIfExists = ifNotExists) + schema, partitioning, tableSpec, ignoreIfExists = ifNotExists, + optionsListExpressions = options) } } @@ -4012,8 +4046,8 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val partitioning = partitionExpressions(partTransforms, partCols, ctx) ++ bucketSpec.map(_.asTransform) - val tableSpec = TableSpec(properties, provider, options, location, comment, - serdeInfo, false) + val tableSpec = UnresolvedTableSpec(properties, provider, location, comment, + serdeInfo, external = false) Option(ctx.query).map(plan) match { case Some(_) if columns.nonEmpty => @@ -4030,7 +4064,8 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit case Some(query) => ReplaceTableAsSelect( withIdentClause(ctx.replaceTableHeader.identifierReference(), UnresolvedIdentifier(_)), - partitioning, query, tableSpec, writeOptions = Map.empty, orCreate = orCreate) + partitioning, query, tableSpec, writeOptions = Map.empty, orCreate = orCreate, + optionsListExpressions = options) case _ => // Note: table schema includes both the table columns list and the partition columns @@ -4038,7 +4073,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val schema = StructType(columns ++ partCols) ReplaceTable( withIdentClause(ctx.replaceTableHeader.identifierReference(), UnresolvedIdentifier(_)), - schema, partitioning, tableSpec, orCreate = orCreate) + schema, partitioning, tableSpec, orCreate = orCreate, optionsListExpressions = options) } } @@ -4073,15 +4108,12 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * Create a [[ShowTables]] command. */ override def visitShowTables(ctx: ShowTablesContext): LogicalPlan = withOrigin(ctx) { - if (ctx.identifierReference() != null) { - ShowTables( - withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)), - Option(ctx.pattern).map(x => string(visitStringLit(x)))) + val ns = if (ctx.identifierReference() != null) { + withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) } else { - ShowTables( - UnresolvedNamespace(Seq.empty[String]), - Option(ctx.pattern).map(x => string(visitStringLit(x)))) + UnresolvedNamespace(Seq.empty[String]) } + ShowTables(ns, Option(ctx.pattern).map(x => string(visitStringLit(x)))) } /** @@ -4092,32 +4124,24 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val partitionKeys = Option(ctx.partitionSpec).map { specCtx => UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(specCtx), None) } - if (ctx.identifierReference() != null) { - ShowTableExtended( - withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)), - string(visitStringLit(ctx.pattern)), - partitionKeys) + val ns = if (ctx.identifierReference() != null) { + withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) } else { - ShowTableExtended( - UnresolvedNamespace(Seq.empty[String]), - string(visitStringLit(ctx.pattern)), - partitionKeys) + UnresolvedNamespace(Seq.empty[String]) } + ShowTableExtended(ns, string(visitStringLit(ctx.pattern)), partitionKeys) } /** * Create a [[ShowViews]] command. */ override def visitShowViews(ctx: ShowViewsContext): LogicalPlan = withOrigin(ctx) { - if (ctx.identifierReference() != null) { - ShowViews( - withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)), - Option(ctx.pattern).map(x => string(visitStringLit(x)))) + val ns = if (ctx.identifierReference() != null) { + withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) } else { - ShowViews( - UnresolvedNamespace(Seq.empty[String]), - Option(ctx.pattern).map(x => string(visitStringLit(x)))) + UnresolvedNamespace(Seq.empty[String]) } + ShowViews(ns, Option(ctx.pattern).map(x => string(visitStringLit(x)))) } override def visitColPosition(ctx: ColPositionContext): ColumnPosition = { @@ -4578,15 +4602,12 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit ctx.identifier.getText.toLowerCase(Locale.ROOT) != "noscan") { throw QueryParsingErrors.computeStatisticsNotExpectedError(ctx.identifier()) } - if (ctx.identifierReference() != null) { - AnalyzeTables( - withIdentClause( - ctx.identifierReference, - UnresolvedNamespace(_)), - noScan = ctx.identifier != null) + val ns = if (ctx.identifierReference() != null) { + withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) } else { - AnalyzeTables(UnresolvedNamespace(Seq.empty[String]), noScan = ctx.identifier != null) + UnresolvedNamespace(Seq.empty[String]) } + AnalyzeTables(ns, noScan = ctx.identifier != null) } /** @@ -4659,19 +4680,17 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val query = Option(ctx.query).map(plan) withIdentClause(ctx.identifierReference, ident => { - val relation = createUnresolvedRelation(ctx.identifierReference, ident) - val tableName = relation.multipartIdentifier - if (query.isDefined && tableName.length > 1) { - val catalogAndNamespace = tableName.init + if (query.isDefined && ident.length > 1) { + val catalogAndNamespace = ident.init throw QueryParsingErrors.addCatalogInCacheTableAsSelectNotAllowedError( catalogAndNamespace.quoted, ctx) } val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) val isLazy = ctx.LAZY != null if (query.isDefined) { - CacheTableAsSelect(tableName.head, query.get, source(ctx.query()), isLazy, options) + CacheTableAsSelect(ident.head, query.get, source(ctx.query()), isLazy, options) } else { - CacheTable(relation, tableName, isLazy, options) + CacheTable(createUnresolvedRelation(ctx.identifierReference, ident), ident, isLazy, options) } }) } @@ -4680,11 +4699,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit * Create an [[UncacheTable]] logical plan. */ override def visitUncacheTable(ctx: UncacheTableContext): LogicalPlan = withOrigin(ctx) { - UncacheTable( - withIdentClause( - ctx.identifierReference, - createUnresolvedRelation(ctx.identifierReference, _)), - ctx.EXISTS != null) + UncacheTable(createUnresolvedRelation(ctx.identifierReference), ctx.EXISTS != null) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index 986e7534a1eba..81bea1d0e12e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -25,8 +25,8 @@ import java.util.{Locale, Properties, ServiceConfigurationError} import org.apache.hadoop.fs.{LocalFileSystem, Path} import org.apache.hadoop.fs.permission.FsPermission import org.mockito.Mockito.{mock, spy, when} - import org.apache.spark._ + import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{Parameter, UnresolvedGenerator} diff --git a/weird_changes.diff b/weird_changes.diff new file mode 100644 index 0000000000000..6cb92e395f89e --- /dev/null +++ b/weird_changes.diff @@ -0,0 +1,586 @@ +commit 20b9b9350b650eda7601e49249a1780b32cd3f07 +Author: Richard Yu +Date: Mon Jun 12 10:36:47 2023 -0700 + + Adding error analysis test + +diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala +index c6b3f99c20d..20f7f30a280 100644 +--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala ++++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala +@@ -25,9 +25,6 @@ import org.apache.spark.sql.types.DataType + * SQL Syntax: key => value + * SQL grammar: key=identifier FAT_ARROW value=expression + * +- * NamedArgumentExpression is expected to be resolved +- * and replaced in class extending [[NamedArgumentFunction]] +- * + * Example usage in encode: + * SELECT encode("abc", charset => "utf-8"); + * SELECT encode(charset => "utf-8", value => "abc"); +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 377f83367ff..64e0b9a0985 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 +@@ -45,7 +45,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, con + import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces, TableCatalog} + import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition + import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} +-import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryParsingErrors} ++import org.apache.spark.sql.errors.QueryParsingErrors + import org.apache.spark.sql.internal.SQLConf + import org.apache.spark.sql.types._ + import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +@@ -277,10 +277,10 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + + /** + * Parameters used for writing query to a table: +- * (table ident, tableColumnList, partitionKeys, ifPartitionNotExists, byName). ++ * (UnresolvedRelation, tableColumnList, partitionKeys, ifPartitionNotExists). + */ +- type InsertTableParams = +- (IdentifierReferenceContext, Seq[String], Map[String, Option[String]], Boolean, Boolean) ++ type InsertTableParams = (IdentifierReferenceContext, ++ Seq[String], Map[String, Option[String]], Boolean) + + /** + * Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat, provider). +@@ -291,7 +291,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + * Add an + * {{{ + * INSERT OVERWRITE TABLE tableIdentifier [partitionSpec [IF NOT EXISTS]]? [identifierList] +- * INSERT INTO [TABLE] tableIdentifier [partitionSpec] ([BY NAME] | [identifierList]) ++ * INSERT INTO [TABLE] tableIdentifier [partitionSpec] [identifierList] + * INSERT INTO [TABLE] tableIdentifier REPLACE whereClause + * INSERT OVERWRITE [LOCAL] DIRECTORY STRING [rowFormat] [createFileFormat] + * INSERT OVERWRITE [LOCAL] DIRECTORY [STRING] tableProvider [OPTIONS tablePropertyList] +@@ -302,29 +302,25 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + ctx: InsertIntoContext, + query: LogicalPlan): LogicalPlan = withOrigin(ctx) { + ctx match { +- // We cannot push withIdentClause() into the write command because: +- // 1. `PlanWithUnresolvedIdentifier` is not a NamedRelation +- // 2. Write commands do not hold the table logical plan as a child, and we need to add +- // additional resolution code to resolve identifiers inside the write commands. + case table: InsertIntoTableContext => +- val (relationCtx, cols, partition, ifPartitionNotExists, byName) +- = visitInsertIntoTable(table) +- withIdentClause(relationCtx, ident => { ++ val (relation, cols, partition, ifPartitionNotExists) = visitInsertIntoTable(table) ++ // We cannot push withIdentClause() into the InsertIntoStatement because ++ // InsertIntoStatement() is a unary node. Changing that two binary will bypass streaming ++ // specific code in InsertIntoStatement resolution in the analyzer ++ withIdentClause(relation, ident => { + InsertIntoStatement( +- createUnresolvedRelation(relationCtx, ident), ++ createUnresolvedRelation(relation, ident), + partition, + cols, + query, + overwrite = false, +- ifPartitionNotExists, +- byName) ++ ifPartitionNotExists) + }) + case table: InsertOverwriteTableContext => +- val (relationCtx, cols, partition, ifPartitionNotExists, _) +- = visitInsertOverwriteTable(table) +- withIdentClause(relationCtx, ident => { ++ val (relation, cols, partition, ifPartitionNotExists) = visitInsertOverwriteTable(table) ++ withIdentClause(relation, ident => { + InsertIntoStatement( +- createUnresolvedRelation(relationCtx, ident), ++ createUnresolvedRelation(relation, ident), + partition, + cols, + query, +@@ -361,7 +357,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + operationNotAllowed("INSERT INTO ... IF NOT EXISTS", ctx) + } + +- (ctx.identifierReference, cols, partitionKeys, false, ctx.NAME() != null) ++ (ctx.identifierReference(), cols, partitionKeys, false) + } + + /** +@@ -379,7 +375,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + dynamicPartitionKeys.keys.mkString(", "), ctx) + } + +- (ctx.identifierReference, cols, partitionKeys, ctx.EXISTS() != null, false) ++ (ctx.identifierReference(), cols, partitionKeys, ctx.EXISTS() != null) + } + + /** +@@ -413,7 +409,9 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + + override def visitDeleteFromTable( + ctx: DeleteFromTableContext): LogicalPlan = withOrigin(ctx) { +- val table = createUnresolvedRelation(ctx.identifierReference) ++ val table = withIdentClause( ++ ctx.identifierReference, ++ createUnresolvedRelation(ctx.identifierReference, _)) + val tableAlias = getTableAliasWithoutColumnAlias(ctx.tableAlias(), "DELETE") + val aliasedTable = tableAlias.map(SubqueryAlias(_, table)).getOrElse(table) + val predicate = if (ctx.whereClause() != null) { +@@ -425,7 +423,9 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + } + + override def visitUpdateTable(ctx: UpdateTableContext): LogicalPlan = withOrigin(ctx) { +- val table = createUnresolvedRelation(ctx.identifierReference) ++ val table = withIdentClause( ++ ctx.identifierReference, ++ createUnresolvedRelation(ctx.identifierReference, _)) + val tableAlias = getTableAliasWithoutColumnAlias(ctx.tableAlias(), "UPDATE") + val aliasedTable = tableAlias.map(SubqueryAlias(_, table)).getOrElse(table) + val assignments = withAssignments(ctx.setClause().assignmentList()) +@@ -447,12 +447,12 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + } + + override def visitMergeIntoTable(ctx: MergeIntoTableContext): LogicalPlan = withOrigin(ctx) { +- val targetTable = createUnresolvedRelation(ctx.target) ++ val targetTable = withIdentClause(ctx.target, createUnresolvedRelation(ctx.target, _ )) + val targetTableAlias = getTableAliasWithoutColumnAlias(ctx.targetAlias, "MERGE") + val aliasedTarget = targetTableAlias.map(SubqueryAlias(_, targetTable)).getOrElse(targetTable) + + val sourceTableOrQuery = if (ctx.source != null) { +- createUnresolvedRelation(ctx.source) ++ withIdentClause(ctx.source, createUnresolvedRelation(ctx.source, _)) + } else if (ctx.sourceQuery != null) { + visitQuery(ctx.sourceQuery) + } else { +@@ -1488,14 +1488,14 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + * }}} + */ + override def visitTable(ctx: TableContext): LogicalPlan = withOrigin(ctx) { +- createUnresolvedRelation(ctx.identifierReference) ++ withIdentClause(ctx.identifierReference, UnresolvedRelation(_)) + } + + /** + * Create an aliased table reference. This is typically used in FROM clauses. + */ + override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) { +- val relation = createUnresolvedRelation(ctx.identifierReference) ++ val relation = withIdentClause(ctx.identifierReference, UnresolvedRelation(_)) + val table = mayApplyAliasPlan( + ctx.tableAlias, relation.optionalMap(ctx.temporalClause)(withTimeTravel)) + table.optionalMap(ctx.sample)(withSample) +@@ -1536,31 +1536,26 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + } else { + Seq.empty + } +- +- withIdentClause( +- func.functionName.expression, +- () => getFunctionMultiparts(func.functionName), +- name => { +- if (name.length > 1) { +- throw QueryParsingErrors.invalidTableValuedFunctionNameError(name, ctx) +- } +- val args = func.functionArgument.asScala.map { e => +- if (e.namedArgumentExpression != null) { +- val key = e.namedArgumentExpression.key.strictIdentifier +- val value = e.namedArgumentExpression.value +- NamedArgumentExpression(key.getText, expression(value)) +- } else { +- expression(e) +- } ++ withFuncIdentClause(func.functionName, name => { ++ if (name.length > 1) { ++ throw QueryParsingErrors.invalidTableValuedFunctionNameError(name, ctx) ++ } ++ val args = func.functionArgument.asScala.map { e => ++ if (e.namedArgumentExpression != null) { ++ val key = e.namedArgumentExpression.key.strictIdentifier ++ val value = e.namedArgumentExpression.value ++ NamedArgumentExpression(key.getText, expression(value)) ++ } else { ++ expression(e) + } ++ } + +- val tvf = UnresolvedTableValuedFunction(name, args) ++ val tvf = UnresolvedTableValuedFunction(name, args) + +- val tvfAliases = if (aliases.nonEmpty) UnresolvedTVFAliases(name, tvf, aliases) else tvf ++ val tvfAliases = if (aliases.nonEmpty) UnresolvedTVFAliases(name, tvf, aliases) else tvf + +- tvfAliases.optionalMap(func.tableAlias.strictIdentifier)(aliasPlan) +- } +- ) ++ tvfAliases.optionalMap(func.tableAlias.strictIdentifier)(aliasPlan) ++ }) + } + + /** +@@ -2171,10 +2166,12 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + } + + /** +- * Create an expression for the IDENTIFIER() clause. ++ * Create a IDENTIFIER() clause. + */ + override def visitIdentifierClause(ctx: IdentifierClauseContext): Expression = withOrigin(ctx) { +- ExpressionWithUnresolvedIdentifier(expression(ctx.expression), UnresolvedAttribute(_)) ++ // Create the function call. ++ val expr: Expression = expression(ctx.expression()) ++ UnresolvedAttributeIdentifierClause(expr) + } + + /** +@@ -2204,18 +2201,19 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + val filter = Option(ctx.where).map(expression(_)) + val ignoreNulls = + Option(ctx.nullsOption).map(_.getType == SqlBaseParser.IGNORE).getOrElse(false) +- val funcCtx = ctx.functionName +- val func = withIdentClause(funcCtx.expression, () => getFunctionMultiparts(funcCtx), ident => { +- UnresolvedFunction(ident, arguments, isDistinct, filter, ignoreNulls) +- }) ++ val function = Option(ctx.functionName.expression()).map(p => ++ UnresolvedFunctionIdentifierClause(expression(p), ++ arguments, isDistinct, filter, ignoreNulls)).getOrElse( ++ UnresolvedFunction(getFunctionMultiparts(ctx.functionName), ++ arguments, isDistinct, filter, ignoreNulls)) + + // Check if the function is evaluated in a windowed context. + ctx.windowSpec match { + case spec: WindowRefContext => +- UnresolvedWindowExpression(func, visitWindowRef(spec)) ++ UnresolvedWindowExpression(function, visitWindowRef(spec)) + case spec: WindowDefContext => +- WindowExpression(func, visitWindowDef(spec)) +- case _ => func ++ WindowExpression(function, visitWindowDef(spec)) ++ case _ => function + } + } + +@@ -2719,14 +2717,6 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + } + } + +- /** +- * Create an [[UnresolvedRelation]] from an identifier reference. +- */ +- private def createUnresolvedRelation( +- ctx: IdentifierReferenceContext): LogicalPlan = withOrigin(ctx) { +- withIdentClause(ctx, UnresolvedRelation(_)) +- } +- + /** + * Create an [[UnresolvedRelation]] from a multi-part identifier. + */ +@@ -2770,7 +2760,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + ctx: ParserRuleContext, + ident: Seq[String], + commandName: String, +- allowTempView: Boolean): UnresolvedTableOrView = withOrigin(ctx) { ++ allowTempView: Boolean ): UnresolvedTableOrView = withOrigin(ctx) { + UnresolvedTableOrView(ident, commandName, allowTempView) + } + +@@ -3360,20 +3350,6 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + } + } + +- /** +- * Parse a key-value map from an [[ExpressionPropertyListContext]], assuming all values are +- * specified. +- */ +- override def visitExpressionPropertyList( +- ctx: ExpressionPropertyListContext): OptionsListExpressions = { +- val options = ctx.expressionProperty.asScala.map { property => +- val key: String = visitPropertyKey(property.key) +- val value: Expression = Option(property.value).map(expression).getOrElse(null) +- key -> value +- }.toSeq +- OptionsListExpressions(options) +- } +- + override def visitStringLit(ctx: StringLitContext): Token = { + if (ctx != null) { + if (ctx.STRING_LITERAL != null) { +@@ -3408,7 +3384,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + */ + type TableClauses = ( + Seq[Transform], Seq[StructField], Option[BucketSpec], Map[String, String], +- OptionsListExpressions, Option[String], Option[String], Option[SerdeInfo]) ++ Map[String, String], Option[String], Option[String], Option[SerdeInfo]) + + /** + * Validate a create table statement and return the [[TableIdentifier]]. +@@ -3668,8 +3644,8 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + ctx.EXTENDED != null) + } + +- def cleanTableProperties[ValueType]( +- ctx: ParserRuleContext, properties: Map[String, ValueType]): Map[String, ValueType] = { ++ def cleanTableProperties( ++ ctx: ParserRuleContext, properties: Map[String, String]): Map[String, String] = { + import TableCatalog._ + val legacyOn = conf.getConf(SQLConf.LEGACY_PROPERTY_NON_RESERVED) + properties.filter { +@@ -3703,26 +3679,18 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + + def cleanTableOptions( + ctx: ParserRuleContext, +- options: OptionsListExpressions, +- location: Option[String]): (OptionsListExpressions, Option[String]) = { ++ options: Map[String, String], ++ location: Option[String]): (Map[String, String], Option[String]) = { + var path = location +- val filtered = cleanTableProperties(ctx, options.options.toMap).filter { +- case (key, value) if key.equalsIgnoreCase("path") => +- val newValue: String = +- if (value == null) { +- "" +- } else value match { +- case Literal(_, _: StringType) => value.toString +- case _ => throw QueryCompilationErrors.optionMustBeLiteralString(key) +- } +- if (path.nonEmpty) { +- throw QueryParsingErrors.duplicatedTablePathsFoundError(path.get, newValue, ctx) +- } +- path = Some(newValue) ++ val filtered = cleanTableProperties(ctx, options).filter { ++ case (k, v) if k.equalsIgnoreCase("path") && path.nonEmpty => ++ throw QueryParsingErrors.duplicatedTablePathsFoundError(path.get, v, ctx) ++ case (k, v) if k.equalsIgnoreCase("path") => ++ path = Some(v) + false + case _ => true + } +- (OptionsListExpressions(filtered.toSeq), path) ++ (filtered, path) + } + + /** +@@ -3880,8 +3848,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + val bucketSpec = ctx.bucketSpec().asScala.headOption.map(visitBucketSpec) + val properties = Option(ctx.tableProps).map(visitPropertyKeyValues).getOrElse(Map.empty) + val cleanedProperties = cleanTableProperties(ctx, properties) +- val options = Option(ctx.options).map(visitExpressionPropertyList) +- .getOrElse(OptionsListExpressions(Seq.empty)) ++ val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) + val location = visitLocationSpecList(ctx.locationSpec()) + val (cleanedOptions, newLocation) = cleanTableOptions(ctx, options, location) + val comment = visitCommentSpecList(ctx.commentSpec()) +@@ -3961,8 +3928,8 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + val columns = Option(ctx.createOrReplaceTableColTypeList()) + .map(visitCreateOrReplaceTableColTypeList).getOrElse(Nil) + val provider = Option(ctx.tableProvider).map(_.multipartIdentifier.getText) +- val (partTransforms, partCols, bucketSpec, properties, options, location, +- comment, serdeInfo) = visitCreateTableClauses(ctx.createTableClauses()) ++ val (partTransforms, partCols, bucketSpec, properties, options, location, comment, serdeInfo) = ++ visitCreateTableClauses(ctx.createTableClauses()) + + if (provider.isDefined && serdeInfo.isDefined) { + operationNotAllowed(s"CREATE TABLE ... USING ... ${serdeInfo.get.describe}", ctx) +@@ -3976,7 +3943,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + + val partitioning = + partitionExpressions(partTransforms, partCols, ctx) ++ bucketSpec.map(_.asTransform) +- val tableSpec = UnresolvedTableSpec(properties, provider, location, comment, ++ val tableSpec = TableSpec(properties, provider, options, location, comment, + serdeInfo, external) + + Option(ctx.query).map(plan) match { +@@ -3993,15 +3960,14 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + + case Some(query) => + CreateTableAsSelect(withIdentClause(identifierContext, UnresolvedIdentifier(_)), +- partitioning, query, tableSpec, Map.empty, ifNotExists, optionsListExpressions = options) ++ partitioning, query, tableSpec, Map.empty, ifNotExists) + + case _ => + // Note: table schema includes both the table columns list and the partition columns + // with data type. + val schema = StructType(columns ++ partCols) + CreateTable(withIdentClause(identifierContext, UnresolvedIdentifier(_)), +- schema, partitioning, tableSpec, ignoreIfExists = ifNotExists, +- optionsListExpressions = options) ++ schema, partitioning, tableSpec, ignoreIfExists = ifNotExists) + } + } + +@@ -4046,8 +4012,8 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + + val partitioning = + partitionExpressions(partTransforms, partCols, ctx) ++ bucketSpec.map(_.asTransform) +- val tableSpec = UnresolvedTableSpec(properties, provider, location, comment, +- serdeInfo, external = false) ++ val tableSpec = TableSpec(properties, provider, options, location, comment, ++ serdeInfo, false) + + Option(ctx.query).map(plan) match { + case Some(_) if columns.nonEmpty => +@@ -4064,8 +4030,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + case Some(query) => + ReplaceTableAsSelect( + withIdentClause(ctx.replaceTableHeader.identifierReference(), UnresolvedIdentifier(_)), +- partitioning, query, tableSpec, writeOptions = Map.empty, orCreate = orCreate, +- optionsListExpressions = options) ++ partitioning, query, tableSpec, writeOptions = Map.empty, orCreate = orCreate) + + case _ => + // Note: table schema includes both the table columns list and the partition columns +@@ -4073,7 +4038,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + val schema = StructType(columns ++ partCols) + ReplaceTable( + withIdentClause(ctx.replaceTableHeader.identifierReference(), UnresolvedIdentifier(_)), +- schema, partitioning, tableSpec, orCreate = orCreate, optionsListExpressions = options) ++ schema, partitioning, tableSpec, orCreate = orCreate) + } + } + +@@ -4108,12 +4073,15 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + * Create a [[ShowTables]] command. + */ + override def visitShowTables(ctx: ShowTablesContext): LogicalPlan = withOrigin(ctx) { +- val ns = if (ctx.identifierReference() != null) { +- withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) ++ if (ctx.identifierReference() != null) { ++ ShowTables( ++ withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)), ++ Option(ctx.pattern).map(x => string(visitStringLit(x)))) + } else { +- UnresolvedNamespace(Seq.empty[String]) ++ ShowTables( ++ UnresolvedNamespace(Seq.empty[String]), ++ Option(ctx.pattern).map(x => string(visitStringLit(x)))) + } +- ShowTables(ns, Option(ctx.pattern).map(x => string(visitStringLit(x)))) + } + + /** +@@ -4124,24 +4092,32 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + val partitionKeys = Option(ctx.partitionSpec).map { specCtx => + UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(specCtx), None) + } +- val ns = if (ctx.identifierReference() != null) { +- withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) ++ if (ctx.identifierReference() != null) { ++ ShowTableExtended( ++ withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)), ++ string(visitStringLit(ctx.pattern)), ++ partitionKeys) + } else { +- UnresolvedNamespace(Seq.empty[String]) ++ ShowTableExtended( ++ UnresolvedNamespace(Seq.empty[String]), ++ string(visitStringLit(ctx.pattern)), ++ partitionKeys) + } +- ShowTableExtended(ns, string(visitStringLit(ctx.pattern)), partitionKeys) + } + + /** + * Create a [[ShowViews]] command. + */ + override def visitShowViews(ctx: ShowViewsContext): LogicalPlan = withOrigin(ctx) { +- val ns = if (ctx.identifierReference() != null) { +- withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) ++ if (ctx.identifierReference() != null) { ++ ShowViews( ++ withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)), ++ Option(ctx.pattern).map(x => string(visitStringLit(x)))) + } else { +- UnresolvedNamespace(Seq.empty[String]) ++ ShowViews( ++ UnresolvedNamespace(Seq.empty[String]), ++ Option(ctx.pattern).map(x => string(visitStringLit(x)))) + } +- ShowViews(ns, Option(ctx.pattern).map(x => string(visitStringLit(x)))) + } + + override def visitColPosition(ctx: ColPositionContext): ColumnPosition = { +@@ -4602,12 +4578,15 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + ctx.identifier.getText.toLowerCase(Locale.ROOT) != "noscan") { + throw QueryParsingErrors.computeStatisticsNotExpectedError(ctx.identifier()) + } +- val ns = if (ctx.identifierReference() != null) { +- withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) ++ if (ctx.identifierReference() != null) { ++ AnalyzeTables( ++ withIdentClause( ++ ctx.identifierReference, ++ UnresolvedNamespace(_)), ++ noScan = ctx.identifier != null) + } else { +- UnresolvedNamespace(Seq.empty[String]) ++ AnalyzeTables(UnresolvedNamespace(Seq.empty[String]), noScan = ctx.identifier != null) + } +- AnalyzeTables(ns, noScan = ctx.identifier != null) + } + + /** +@@ -4680,17 +4659,19 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + + val query = Option(ctx.query).map(plan) + withIdentClause(ctx.identifierReference, ident => { +- if (query.isDefined && ident.length > 1) { +- val catalogAndNamespace = ident.init ++ val relation = createUnresolvedRelation(ctx.identifierReference, ident) ++ val tableName = relation.multipartIdentifier ++ if (query.isDefined && tableName.length > 1) { ++ val catalogAndNamespace = tableName.init + throw QueryParsingErrors.addCatalogInCacheTableAsSelectNotAllowedError( + catalogAndNamespace.quoted, ctx) + } + val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) + val isLazy = ctx.LAZY != null + if (query.isDefined) { +- CacheTableAsSelect(ident.head, query.get, source(ctx.query()), isLazy, options) ++ CacheTableAsSelect(tableName.head, query.get, source(ctx.query()), isLazy, options) + } else { +- CacheTable(createUnresolvedRelation(ctx.identifierReference, ident), ident, isLazy, options) ++ CacheTable(relation, tableName, isLazy, options) + } + }) + } +@@ -4699,7 +4680,11 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit + * Create an [[UncacheTable]] logical plan. + */ + override def visitUncacheTable(ctx: UncacheTableContext): LogicalPlan = withOrigin(ctx) { +- UncacheTable(createUnresolvedRelation(ctx.identifierReference), ctx.EXISTS != null) ++ UncacheTable( ++ withIdentClause( ++ ctx.identifierReference, ++ createUnresolvedRelation(ctx.identifierReference, _)), ++ ctx.EXISTS != null) + } + + /** +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +index 069fce237f2..986e7534a1e 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +@@ -30,7 +30,7 @@ import org.apache.spark._ + import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, Row, SaveMode} + import org.apache.spark.sql.catalyst.FunctionIdentifier + import org.apache.spark.sql.catalyst.analysis.{Parameter, UnresolvedGenerator} +-import org.apache.spark.sql.catalyst.expressions.{Grouping, Literal, RowNumber} ++import org.apache.spark.sql.catalyst.expressions.{Grouping, Literal, NamedArgumentExpression, RowNumber} + import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ + import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext + import org.apache.spark.sql.catalyst.expressions.objects.InitializeJavaBean +@@ -837,6 +837,18 @@ class QueryExecutionErrorsSuite + sqlState = "XX000") + } + ++ test("INTERNAL_ERROR: Calling eval on Unevaluable NamedArgumentExpression") { ++ val e = intercept[SparkException] { ++ NamedArgumentExpression("arg0", Literal("value0")).eval() ++ } ++ checkError( ++ exception = e, ++ errorClass = "INTERNAL_ERROR", ++ parameters = Map("message" -> "Cannot evaluate expression: arg0 => value0"), ++ sqlState = "XX000" ++ ) ++ } ++ + test("INTERNAL_ERROR: Calling doGenCode on unresolved") { + val e = intercept[SparkException] { + val ctx = new CodegenContext From 95b3f20b2aa66c28fc0a4a585f08a966a61d06a7 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Mon, 12 Jun 2023 10:48:34 -0700 Subject: [PATCH 07/36] Removing file --- weird_changes.diff | 586 --------------------------------------------- 1 file changed, 586 deletions(-) delete mode 100644 weird_changes.diff diff --git a/weird_changes.diff b/weird_changes.diff deleted file mode 100644 index 6cb92e395f89e..0000000000000 --- a/weird_changes.diff +++ /dev/null @@ -1,586 +0,0 @@ -commit 20b9b9350b650eda7601e49249a1780b32cd3f07 -Author: Richard Yu -Date: Mon Jun 12 10:36:47 2023 -0700 - - Adding error analysis test - -diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala -index c6b3f99c20d..20f7f30a280 100644 ---- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala -+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala -@@ -25,9 +25,6 @@ import org.apache.spark.sql.types.DataType - * SQL Syntax: key => value - * SQL grammar: key=identifier FAT_ARROW value=expression - * -- * NamedArgumentExpression is expected to be resolved -- * and replaced in class extending [[NamedArgumentFunction]] -- * - * Example usage in encode: - * SELECT encode("abc", charset => "utf-8"); - * SELECT encode(charset => "utf-8", value => "abc"); -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 377f83367ff..64e0b9a0985 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 -@@ -45,7 +45,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, con - import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces, TableCatalog} - import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition - import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} --import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryParsingErrors} -+import org.apache.spark.sql.errors.QueryParsingErrors - import org.apache.spark.sql.internal.SQLConf - import org.apache.spark.sql.types._ - import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} -@@ -277,10 +277,10 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - - /** - * Parameters used for writing query to a table: -- * (table ident, tableColumnList, partitionKeys, ifPartitionNotExists, byName). -+ * (UnresolvedRelation, tableColumnList, partitionKeys, ifPartitionNotExists). - */ -- type InsertTableParams = -- (IdentifierReferenceContext, Seq[String], Map[String, Option[String]], Boolean, Boolean) -+ type InsertTableParams = (IdentifierReferenceContext, -+ Seq[String], Map[String, Option[String]], Boolean) - - /** - * Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat, provider). -@@ -291,7 +291,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - * Add an - * {{{ - * INSERT OVERWRITE TABLE tableIdentifier [partitionSpec [IF NOT EXISTS]]? [identifierList] -- * INSERT INTO [TABLE] tableIdentifier [partitionSpec] ([BY NAME] | [identifierList]) -+ * INSERT INTO [TABLE] tableIdentifier [partitionSpec] [identifierList] - * INSERT INTO [TABLE] tableIdentifier REPLACE whereClause - * INSERT OVERWRITE [LOCAL] DIRECTORY STRING [rowFormat] [createFileFormat] - * INSERT OVERWRITE [LOCAL] DIRECTORY [STRING] tableProvider [OPTIONS tablePropertyList] -@@ -302,29 +302,25 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - ctx: InsertIntoContext, - query: LogicalPlan): LogicalPlan = withOrigin(ctx) { - ctx match { -- // We cannot push withIdentClause() into the write command because: -- // 1. `PlanWithUnresolvedIdentifier` is not a NamedRelation -- // 2. Write commands do not hold the table logical plan as a child, and we need to add -- // additional resolution code to resolve identifiers inside the write commands. - case table: InsertIntoTableContext => -- val (relationCtx, cols, partition, ifPartitionNotExists, byName) -- = visitInsertIntoTable(table) -- withIdentClause(relationCtx, ident => { -+ val (relation, cols, partition, ifPartitionNotExists) = visitInsertIntoTable(table) -+ // We cannot push withIdentClause() into the InsertIntoStatement because -+ // InsertIntoStatement() is a unary node. Changing that two binary will bypass streaming -+ // specific code in InsertIntoStatement resolution in the analyzer -+ withIdentClause(relation, ident => { - InsertIntoStatement( -- createUnresolvedRelation(relationCtx, ident), -+ createUnresolvedRelation(relation, ident), - partition, - cols, - query, - overwrite = false, -- ifPartitionNotExists, -- byName) -+ ifPartitionNotExists) - }) - case table: InsertOverwriteTableContext => -- val (relationCtx, cols, partition, ifPartitionNotExists, _) -- = visitInsertOverwriteTable(table) -- withIdentClause(relationCtx, ident => { -+ val (relation, cols, partition, ifPartitionNotExists) = visitInsertOverwriteTable(table) -+ withIdentClause(relation, ident => { - InsertIntoStatement( -- createUnresolvedRelation(relationCtx, ident), -+ createUnresolvedRelation(relation, ident), - partition, - cols, - query, -@@ -361,7 +357,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - operationNotAllowed("INSERT INTO ... IF NOT EXISTS", ctx) - } - -- (ctx.identifierReference, cols, partitionKeys, false, ctx.NAME() != null) -+ (ctx.identifierReference(), cols, partitionKeys, false) - } - - /** -@@ -379,7 +375,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - dynamicPartitionKeys.keys.mkString(", "), ctx) - } - -- (ctx.identifierReference, cols, partitionKeys, ctx.EXISTS() != null, false) -+ (ctx.identifierReference(), cols, partitionKeys, ctx.EXISTS() != null) - } - - /** -@@ -413,7 +409,9 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - - override def visitDeleteFromTable( - ctx: DeleteFromTableContext): LogicalPlan = withOrigin(ctx) { -- val table = createUnresolvedRelation(ctx.identifierReference) -+ val table = withIdentClause( -+ ctx.identifierReference, -+ createUnresolvedRelation(ctx.identifierReference, _)) - val tableAlias = getTableAliasWithoutColumnAlias(ctx.tableAlias(), "DELETE") - val aliasedTable = tableAlias.map(SubqueryAlias(_, table)).getOrElse(table) - val predicate = if (ctx.whereClause() != null) { -@@ -425,7 +423,9 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - } - - override def visitUpdateTable(ctx: UpdateTableContext): LogicalPlan = withOrigin(ctx) { -- val table = createUnresolvedRelation(ctx.identifierReference) -+ val table = withIdentClause( -+ ctx.identifierReference, -+ createUnresolvedRelation(ctx.identifierReference, _)) - val tableAlias = getTableAliasWithoutColumnAlias(ctx.tableAlias(), "UPDATE") - val aliasedTable = tableAlias.map(SubqueryAlias(_, table)).getOrElse(table) - val assignments = withAssignments(ctx.setClause().assignmentList()) -@@ -447,12 +447,12 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - } - - override def visitMergeIntoTable(ctx: MergeIntoTableContext): LogicalPlan = withOrigin(ctx) { -- val targetTable = createUnresolvedRelation(ctx.target) -+ val targetTable = withIdentClause(ctx.target, createUnresolvedRelation(ctx.target, _ )) - val targetTableAlias = getTableAliasWithoutColumnAlias(ctx.targetAlias, "MERGE") - val aliasedTarget = targetTableAlias.map(SubqueryAlias(_, targetTable)).getOrElse(targetTable) - - val sourceTableOrQuery = if (ctx.source != null) { -- createUnresolvedRelation(ctx.source) -+ withIdentClause(ctx.source, createUnresolvedRelation(ctx.source, _)) - } else if (ctx.sourceQuery != null) { - visitQuery(ctx.sourceQuery) - } else { -@@ -1488,14 +1488,14 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - * }}} - */ - override def visitTable(ctx: TableContext): LogicalPlan = withOrigin(ctx) { -- createUnresolvedRelation(ctx.identifierReference) -+ withIdentClause(ctx.identifierReference, UnresolvedRelation(_)) - } - - /** - * Create an aliased table reference. This is typically used in FROM clauses. - */ - override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) { -- val relation = createUnresolvedRelation(ctx.identifierReference) -+ val relation = withIdentClause(ctx.identifierReference, UnresolvedRelation(_)) - val table = mayApplyAliasPlan( - ctx.tableAlias, relation.optionalMap(ctx.temporalClause)(withTimeTravel)) - table.optionalMap(ctx.sample)(withSample) -@@ -1536,31 +1536,26 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - } else { - Seq.empty - } -- -- withIdentClause( -- func.functionName.expression, -- () => getFunctionMultiparts(func.functionName), -- name => { -- if (name.length > 1) { -- throw QueryParsingErrors.invalidTableValuedFunctionNameError(name, ctx) -- } -- val args = func.functionArgument.asScala.map { e => -- if (e.namedArgumentExpression != null) { -- val key = e.namedArgumentExpression.key.strictIdentifier -- val value = e.namedArgumentExpression.value -- NamedArgumentExpression(key.getText, expression(value)) -- } else { -- expression(e) -- } -+ withFuncIdentClause(func.functionName, name => { -+ if (name.length > 1) { -+ throw QueryParsingErrors.invalidTableValuedFunctionNameError(name, ctx) -+ } -+ val args = func.functionArgument.asScala.map { e => -+ if (e.namedArgumentExpression != null) { -+ val key = e.namedArgumentExpression.key.strictIdentifier -+ val value = e.namedArgumentExpression.value -+ NamedArgumentExpression(key.getText, expression(value)) -+ } else { -+ expression(e) - } -+ } - -- val tvf = UnresolvedTableValuedFunction(name, args) -+ val tvf = UnresolvedTableValuedFunction(name, args) - -- val tvfAliases = if (aliases.nonEmpty) UnresolvedTVFAliases(name, tvf, aliases) else tvf -+ val tvfAliases = if (aliases.nonEmpty) UnresolvedTVFAliases(name, tvf, aliases) else tvf - -- tvfAliases.optionalMap(func.tableAlias.strictIdentifier)(aliasPlan) -- } -- ) -+ tvfAliases.optionalMap(func.tableAlias.strictIdentifier)(aliasPlan) -+ }) - } - - /** -@@ -2171,10 +2166,12 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - } - - /** -- * Create an expression for the IDENTIFIER() clause. -+ * Create a IDENTIFIER() clause. - */ - override def visitIdentifierClause(ctx: IdentifierClauseContext): Expression = withOrigin(ctx) { -- ExpressionWithUnresolvedIdentifier(expression(ctx.expression), UnresolvedAttribute(_)) -+ // Create the function call. -+ val expr: Expression = expression(ctx.expression()) -+ UnresolvedAttributeIdentifierClause(expr) - } - - /** -@@ -2204,18 +2201,19 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - val filter = Option(ctx.where).map(expression(_)) - val ignoreNulls = - Option(ctx.nullsOption).map(_.getType == SqlBaseParser.IGNORE).getOrElse(false) -- val funcCtx = ctx.functionName -- val func = withIdentClause(funcCtx.expression, () => getFunctionMultiparts(funcCtx), ident => { -- UnresolvedFunction(ident, arguments, isDistinct, filter, ignoreNulls) -- }) -+ val function = Option(ctx.functionName.expression()).map(p => -+ UnresolvedFunctionIdentifierClause(expression(p), -+ arguments, isDistinct, filter, ignoreNulls)).getOrElse( -+ UnresolvedFunction(getFunctionMultiparts(ctx.functionName), -+ arguments, isDistinct, filter, ignoreNulls)) - - // Check if the function is evaluated in a windowed context. - ctx.windowSpec match { - case spec: WindowRefContext => -- UnresolvedWindowExpression(func, visitWindowRef(spec)) -+ UnresolvedWindowExpression(function, visitWindowRef(spec)) - case spec: WindowDefContext => -- WindowExpression(func, visitWindowDef(spec)) -- case _ => func -+ WindowExpression(function, visitWindowDef(spec)) -+ case _ => function - } - } - -@@ -2719,14 +2717,6 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - } - } - -- /** -- * Create an [[UnresolvedRelation]] from an identifier reference. -- */ -- private def createUnresolvedRelation( -- ctx: IdentifierReferenceContext): LogicalPlan = withOrigin(ctx) { -- withIdentClause(ctx, UnresolvedRelation(_)) -- } -- - /** - * Create an [[UnresolvedRelation]] from a multi-part identifier. - */ -@@ -2770,7 +2760,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - ctx: ParserRuleContext, - ident: Seq[String], - commandName: String, -- allowTempView: Boolean): UnresolvedTableOrView = withOrigin(ctx) { -+ allowTempView: Boolean ): UnresolvedTableOrView = withOrigin(ctx) { - UnresolvedTableOrView(ident, commandName, allowTempView) - } - -@@ -3360,20 +3350,6 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - } - } - -- /** -- * Parse a key-value map from an [[ExpressionPropertyListContext]], assuming all values are -- * specified. -- */ -- override def visitExpressionPropertyList( -- ctx: ExpressionPropertyListContext): OptionsListExpressions = { -- val options = ctx.expressionProperty.asScala.map { property => -- val key: String = visitPropertyKey(property.key) -- val value: Expression = Option(property.value).map(expression).getOrElse(null) -- key -> value -- }.toSeq -- OptionsListExpressions(options) -- } -- - override def visitStringLit(ctx: StringLitContext): Token = { - if (ctx != null) { - if (ctx.STRING_LITERAL != null) { -@@ -3408,7 +3384,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - */ - type TableClauses = ( - Seq[Transform], Seq[StructField], Option[BucketSpec], Map[String, String], -- OptionsListExpressions, Option[String], Option[String], Option[SerdeInfo]) -+ Map[String, String], Option[String], Option[String], Option[SerdeInfo]) - - /** - * Validate a create table statement and return the [[TableIdentifier]]. -@@ -3668,8 +3644,8 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - ctx.EXTENDED != null) - } - -- def cleanTableProperties[ValueType]( -- ctx: ParserRuleContext, properties: Map[String, ValueType]): Map[String, ValueType] = { -+ def cleanTableProperties( -+ ctx: ParserRuleContext, properties: Map[String, String]): Map[String, String] = { - import TableCatalog._ - val legacyOn = conf.getConf(SQLConf.LEGACY_PROPERTY_NON_RESERVED) - properties.filter { -@@ -3703,26 +3679,18 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - - def cleanTableOptions( - ctx: ParserRuleContext, -- options: OptionsListExpressions, -- location: Option[String]): (OptionsListExpressions, Option[String]) = { -+ options: Map[String, String], -+ location: Option[String]): (Map[String, String], Option[String]) = { - var path = location -- val filtered = cleanTableProperties(ctx, options.options.toMap).filter { -- case (key, value) if key.equalsIgnoreCase("path") => -- val newValue: String = -- if (value == null) { -- "" -- } else value match { -- case Literal(_, _: StringType) => value.toString -- case _ => throw QueryCompilationErrors.optionMustBeLiteralString(key) -- } -- if (path.nonEmpty) { -- throw QueryParsingErrors.duplicatedTablePathsFoundError(path.get, newValue, ctx) -- } -- path = Some(newValue) -+ val filtered = cleanTableProperties(ctx, options).filter { -+ case (k, v) if k.equalsIgnoreCase("path") && path.nonEmpty => -+ throw QueryParsingErrors.duplicatedTablePathsFoundError(path.get, v, ctx) -+ case (k, v) if k.equalsIgnoreCase("path") => -+ path = Some(v) - false - case _ => true - } -- (OptionsListExpressions(filtered.toSeq), path) -+ (filtered, path) - } - - /** -@@ -3880,8 +3848,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - val bucketSpec = ctx.bucketSpec().asScala.headOption.map(visitBucketSpec) - val properties = Option(ctx.tableProps).map(visitPropertyKeyValues).getOrElse(Map.empty) - val cleanedProperties = cleanTableProperties(ctx, properties) -- val options = Option(ctx.options).map(visitExpressionPropertyList) -- .getOrElse(OptionsListExpressions(Seq.empty)) -+ val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) - val location = visitLocationSpecList(ctx.locationSpec()) - val (cleanedOptions, newLocation) = cleanTableOptions(ctx, options, location) - val comment = visitCommentSpecList(ctx.commentSpec()) -@@ -3961,8 +3928,8 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - val columns = Option(ctx.createOrReplaceTableColTypeList()) - .map(visitCreateOrReplaceTableColTypeList).getOrElse(Nil) - val provider = Option(ctx.tableProvider).map(_.multipartIdentifier.getText) -- val (partTransforms, partCols, bucketSpec, properties, options, location, -- comment, serdeInfo) = visitCreateTableClauses(ctx.createTableClauses()) -+ val (partTransforms, partCols, bucketSpec, properties, options, location, comment, serdeInfo) = -+ visitCreateTableClauses(ctx.createTableClauses()) - - if (provider.isDefined && serdeInfo.isDefined) { - operationNotAllowed(s"CREATE TABLE ... USING ... ${serdeInfo.get.describe}", ctx) -@@ -3976,7 +3943,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - - val partitioning = - partitionExpressions(partTransforms, partCols, ctx) ++ bucketSpec.map(_.asTransform) -- val tableSpec = UnresolvedTableSpec(properties, provider, location, comment, -+ val tableSpec = TableSpec(properties, provider, options, location, comment, - serdeInfo, external) - - Option(ctx.query).map(plan) match { -@@ -3993,15 +3960,14 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - - case Some(query) => - CreateTableAsSelect(withIdentClause(identifierContext, UnresolvedIdentifier(_)), -- partitioning, query, tableSpec, Map.empty, ifNotExists, optionsListExpressions = options) -+ partitioning, query, tableSpec, Map.empty, ifNotExists) - - case _ => - // Note: table schema includes both the table columns list and the partition columns - // with data type. - val schema = StructType(columns ++ partCols) - CreateTable(withIdentClause(identifierContext, UnresolvedIdentifier(_)), -- schema, partitioning, tableSpec, ignoreIfExists = ifNotExists, -- optionsListExpressions = options) -+ schema, partitioning, tableSpec, ignoreIfExists = ifNotExists) - } - } - -@@ -4046,8 +4012,8 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - - val partitioning = - partitionExpressions(partTransforms, partCols, ctx) ++ bucketSpec.map(_.asTransform) -- val tableSpec = UnresolvedTableSpec(properties, provider, location, comment, -- serdeInfo, external = false) -+ val tableSpec = TableSpec(properties, provider, options, location, comment, -+ serdeInfo, false) - - Option(ctx.query).map(plan) match { - case Some(_) if columns.nonEmpty => -@@ -4064,8 +4030,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - case Some(query) => - ReplaceTableAsSelect( - withIdentClause(ctx.replaceTableHeader.identifierReference(), UnresolvedIdentifier(_)), -- partitioning, query, tableSpec, writeOptions = Map.empty, orCreate = orCreate, -- optionsListExpressions = options) -+ partitioning, query, tableSpec, writeOptions = Map.empty, orCreate = orCreate) - - case _ => - // Note: table schema includes both the table columns list and the partition columns -@@ -4073,7 +4038,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - val schema = StructType(columns ++ partCols) - ReplaceTable( - withIdentClause(ctx.replaceTableHeader.identifierReference(), UnresolvedIdentifier(_)), -- schema, partitioning, tableSpec, orCreate = orCreate, optionsListExpressions = options) -+ schema, partitioning, tableSpec, orCreate = orCreate) - } - } - -@@ -4108,12 +4073,15 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - * Create a [[ShowTables]] command. - */ - override def visitShowTables(ctx: ShowTablesContext): LogicalPlan = withOrigin(ctx) { -- val ns = if (ctx.identifierReference() != null) { -- withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) -+ if (ctx.identifierReference() != null) { -+ ShowTables( -+ withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)), -+ Option(ctx.pattern).map(x => string(visitStringLit(x)))) - } else { -- UnresolvedNamespace(Seq.empty[String]) -+ ShowTables( -+ UnresolvedNamespace(Seq.empty[String]), -+ Option(ctx.pattern).map(x => string(visitStringLit(x)))) - } -- ShowTables(ns, Option(ctx.pattern).map(x => string(visitStringLit(x)))) - } - - /** -@@ -4124,24 +4092,32 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - val partitionKeys = Option(ctx.partitionSpec).map { specCtx => - UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(specCtx), None) - } -- val ns = if (ctx.identifierReference() != null) { -- withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) -+ if (ctx.identifierReference() != null) { -+ ShowTableExtended( -+ withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)), -+ string(visitStringLit(ctx.pattern)), -+ partitionKeys) - } else { -- UnresolvedNamespace(Seq.empty[String]) -+ ShowTableExtended( -+ UnresolvedNamespace(Seq.empty[String]), -+ string(visitStringLit(ctx.pattern)), -+ partitionKeys) - } -- ShowTableExtended(ns, string(visitStringLit(ctx.pattern)), partitionKeys) - } - - /** - * Create a [[ShowViews]] command. - */ - override def visitShowViews(ctx: ShowViewsContext): LogicalPlan = withOrigin(ctx) { -- val ns = if (ctx.identifierReference() != null) { -- withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) -+ if (ctx.identifierReference() != null) { -+ ShowViews( -+ withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)), -+ Option(ctx.pattern).map(x => string(visitStringLit(x)))) - } else { -- UnresolvedNamespace(Seq.empty[String]) -+ ShowViews( -+ UnresolvedNamespace(Seq.empty[String]), -+ Option(ctx.pattern).map(x => string(visitStringLit(x)))) - } -- ShowViews(ns, Option(ctx.pattern).map(x => string(visitStringLit(x)))) - } - - override def visitColPosition(ctx: ColPositionContext): ColumnPosition = { -@@ -4602,12 +4578,15 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - ctx.identifier.getText.toLowerCase(Locale.ROOT) != "noscan") { - throw QueryParsingErrors.computeStatisticsNotExpectedError(ctx.identifier()) - } -- val ns = if (ctx.identifierReference() != null) { -- withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) -+ if (ctx.identifierReference() != null) { -+ AnalyzeTables( -+ withIdentClause( -+ ctx.identifierReference, -+ UnresolvedNamespace(_)), -+ noScan = ctx.identifier != null) - } else { -- UnresolvedNamespace(Seq.empty[String]) -+ AnalyzeTables(UnresolvedNamespace(Seq.empty[String]), noScan = ctx.identifier != null) - } -- AnalyzeTables(ns, noScan = ctx.identifier != null) - } - - /** -@@ -4680,17 +4659,19 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - - val query = Option(ctx.query).map(plan) - withIdentClause(ctx.identifierReference, ident => { -- if (query.isDefined && ident.length > 1) { -- val catalogAndNamespace = ident.init -+ val relation = createUnresolvedRelation(ctx.identifierReference, ident) -+ val tableName = relation.multipartIdentifier -+ if (query.isDefined && tableName.length > 1) { -+ val catalogAndNamespace = tableName.init - throw QueryParsingErrors.addCatalogInCacheTableAsSelectNotAllowedError( - catalogAndNamespace.quoted, ctx) - } - val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) - val isLazy = ctx.LAZY != null - if (query.isDefined) { -- CacheTableAsSelect(ident.head, query.get, source(ctx.query()), isLazy, options) -+ CacheTableAsSelect(tableName.head, query.get, source(ctx.query()), isLazy, options) - } else { -- CacheTable(createUnresolvedRelation(ctx.identifierReference, ident), ident, isLazy, options) -+ CacheTable(relation, tableName, isLazy, options) - } - }) - } -@@ -4699,7 +4680,11 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit - * Create an [[UncacheTable]] logical plan. - */ - override def visitUncacheTable(ctx: UncacheTableContext): LogicalPlan = withOrigin(ctx) { -- UncacheTable(createUnresolvedRelation(ctx.identifierReference), ctx.EXISTS != null) -+ UncacheTable( -+ withIdentClause( -+ ctx.identifierReference, -+ createUnresolvedRelation(ctx.identifierReference, _)), -+ ctx.EXISTS != null) - } - - /** -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala -index 069fce237f2..986e7534a1e 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala -@@ -30,7 +30,7 @@ import org.apache.spark._ - import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, Row, SaveMode} - import org.apache.spark.sql.catalyst.FunctionIdentifier - import org.apache.spark.sql.catalyst.analysis.{Parameter, UnresolvedGenerator} --import org.apache.spark.sql.catalyst.expressions.{Grouping, Literal, RowNumber} -+import org.apache.spark.sql.catalyst.expressions.{Grouping, Literal, NamedArgumentExpression, RowNumber} - import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ - import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext - import org.apache.spark.sql.catalyst.expressions.objects.InitializeJavaBean -@@ -837,6 +837,18 @@ class QueryExecutionErrorsSuite - sqlState = "XX000") - } - -+ test("INTERNAL_ERROR: Calling eval on Unevaluable NamedArgumentExpression") { -+ val e = intercept[SparkException] { -+ NamedArgumentExpression("arg0", Literal("value0")).eval() -+ } -+ checkError( -+ exception = e, -+ errorClass = "INTERNAL_ERROR", -+ parameters = Map("message" -> "Cannot evaluate expression: arg0 => value0"), -+ sqlState = "XX000" -+ ) -+ } -+ - test("INTERNAL_ERROR: Calling doGenCode on unresolved") { - val e = intercept[SparkException] { - val ctx = new CodegenContext From 83129808d796529fcb58ab9f759d57c307001402 Mon Sep 17 00:00:00 2001 From: Richard Yu <134337791+learningchess2003@users.noreply.github.com> Date: Mon, 12 Jun 2023 10:49:46 -0700 Subject: [PATCH 08/36] Update sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala Co-authored-by: Xinyi --- .../spark/sql/catalyst/expressions/NamedArgumentExpression.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala index 20f7f30a2806a..0f3c5472efc9b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.types.DataType * * Example usage in encode: * SELECT encode("abc", charset => "utf-8"); + The second argument generates a `NamedArgumentExpression("charset", ...... * SELECT encode(charset => "utf-8", value => "abc"); * * @param key The name of the function argument From 30a72e910f7acc9ef306497264746ab7233fddbc Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Mon, 12 Jun 2023 11:01:26 -0700 Subject: [PATCH 09/36] Adding some minor comments --- .../sql/catalyst/expressions/NamedArgumentExpression.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala index 0f3c5472efc9b..fd008899d213d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.types.DataType * * Example usage in encode: * SELECT encode("abc", charset => "utf-8"); - The second argument generates a `NamedArgumentExpression("charset", ...... + * The second argument generates `NamedArgumentExpression("charset", Literal("utf-8")) * SELECT encode(charset => "utf-8", value => "abc"); * * @param key The name of the function argument From 515d153c81c81079f559aaf110ffe1478c88cc8c Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Mon, 12 Jun 2023 11:02:13 -0700 Subject: [PATCH 10/36] Comment --- .../sql/catalyst/expressions/NamedArgumentExpression.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala index fd008899d213d..345534989e6d3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.types.DataType * * Example usage in encode: * SELECT encode("abc", charset => "utf-8"); - * The second argument generates `NamedArgumentExpression("charset", Literal("utf-8")) + * The second argument generates NamedArgumentExpression("charset", Literal("utf-8")) * SELECT encode(charset => "utf-8", value => "abc"); * * @param key The name of the function argument From 7a8f4ad14f41ce5025b7bfdd584584082987433d Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Mon, 12 Jun 2023 14:54:37 -0700 Subject: [PATCH 11/36] Adding suggested tests --- .../expressions/NamedArgumentExpression.scala | 2 +- .../parser/ExpressionParserSuite.scala | 10 ++++++++++ .../sql/catalyst/parser/PlanParserSuite.scala | 5 +++++ .../errors/QueryExecutionErrorsSuite.scala | 2 +- .../sql/errors/QueryParsingErrorsSuite.scala | 19 +++++++++++++++++++ 5 files changed, 36 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala index 345534989e6d3..b02e5ade2b09b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala @@ -39,7 +39,7 @@ case class NamedArgumentExpression(key: String, value: Expression) override def dataType: DataType = value.dataType - override def toString: String = s"""$key => $value""" + override def toString: String = s"$key => $value" // NamedArgumentExpression has a single child, which is its value expression, // so the value expression can be resolved by Analyzer rules recursively. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 098ce5b8c1b84..36ea89e2fdd75 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} +import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Project} import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, IntervalUtils} import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.internal.SQLConf @@ -339,6 +340,15 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("encode(charset => 'utf-8', 'abc')", $"encode".function( NamedArgumentExpression("charset", Literal("utf-8")), Literal("abc"))) + assertEqual("encode('abc', charset => 'utf' || '-8')", + $"encode".function(Literal("abc"), + NamedArgumentExpression("charset", + Concat(Literal("utf") :: Literal("-8") :: Nil)))) + val unresolvedAlias = Project(Seq(UnresolvedAlias(Literal("1"))), OneRowRelation()) + assertEqual("encode(value => ((select '1')), charset => 'utf-8')", + $"encode".function( + NamedArgumentExpression("value", ScalarSubquery(plan = unresolvedAlias)), + NamedArgumentExpression("charset", Literal("utf-8")))) } private def lv(s: Symbol) = UnresolvedNamedLambdaVariable(Seq(s.name)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 77675bb27bc0c..6faa84a69c3bf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -1435,6 +1435,11 @@ class PlanParserSuite extends AnalysisTest { NamedArgumentExpression("arg1", Literal("value1")) :: Literal(2) :: NamedArgumentExpression("arg2", Literal(true)) :: Nil).select(star())) + + assertEqual( + "select * from my_tvf(group => 'abc')", + UnresolvedTableValuedFunction("my_tvf", + NamedArgumentExpression("group", Literal("abc")) :: Nil).select(star())) } test("SPARK-32106: TRANSFORM plan") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index 81bea1d0e12e0..986e7534a1eba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -25,8 +25,8 @@ import java.util.{Locale, Properties, ServiceConfigurationError} import org.apache.hadoop.fs.{LocalFileSystem, Path} import org.apache.hadoop.fs.permission.FsPermission import org.mockito.Mockito.{mock, spy, when} -import org.apache.spark._ +import org.apache.spark._ import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{Parameter, UnresolvedGenerator} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala index 58e3fefc8bf0a..1be5f6469a5ad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala @@ -368,6 +368,25 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession { parameters = Map("error" -> "end of input", "hint" -> "")) } + def checkParseSyntaxError(sqlCommand: String, errorString: String, hint: String = ""): Unit = { + checkError( + exception = parseException(sqlCommand), + errorClass = "PARSE_SYNTAX_ERROR", + sqlState = "42601", + parameters = Map("error" -> errorString, "hint" -> hint) + ) + } + + test("PARSE_SYNTAX_ERROR: named arguments invalid syntax") { + checkParseSyntaxError("select * from my_tvf(arg1 ==> 'value1')", "'>'") + checkParseSyntaxError("select * from my_tvf(arg1 = => 'value1')", "'=>'") + checkParseSyntaxError("select * from my_tvf((arg1 => 'value1'))", "'=>'") + checkParseSyntaxError("select * from my_tvf(arg1 => )", "')'") + checkParseSyntaxError("select * from my_tvf(arg1 => , 42)", "','") + checkParseSyntaxError("select * from my_tvf(my_tvf.arg1 => 'value1')", "'=>'") + checkParseSyntaxError("select * from my_tvf(arg1 => table t1)", "'t1'", hint = ": extra input 't1'") + } + test("PARSE_SYNTAX_ERROR: extraneous input") { checkError( exception = parseException("select 1 1"), From 23f9c98d38dfc3d0fef8521523fbbc65d78597b6 Mon Sep 17 00:00:00 2001 From: Richard Yu <134337791+learningchess2003@users.noreply.github.com> Date: Mon, 12 Jun 2023 16:14:11 -0700 Subject: [PATCH 12/36] Update sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala Co-authored-by: Daniel Tenedorio --- .../catalyst/expressions/NamedArgumentExpression.scala | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala index b02e5ade2b09b..6b4269f45f419 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala @@ -20,7 +20,14 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.types.DataType /** - * An unevaluable unary expression specifically for named argument functions + * This represents an argument expression to a function call accompanied with an + * explicit reference to the corresponding argument name as a string. In this way, + * the analyzer can make sure that the provided values match up to the arguments + * as intended, and the arguments may appear in any order. + * This unary expression is unevaluable because we intend to replace it with + * the provided value itself during query analysis (after possibly rearranging + * the parsed argument list to match up the names to the expected function + * signature). * * SQL Syntax: key => value * SQL grammar: key=identifier FAT_ARROW value=expression From f53dab70e780a4f7330c70edcc4fb86f61917d73 Mon Sep 17 00:00:00 2001 From: Richard Yu <134337791+learningchess2003@users.noreply.github.com> Date: Mon, 12 Jun 2023 16:14:22 -0700 Subject: [PATCH 13/36] Update sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala Co-authored-by: Daniel Tenedorio --- .../sql/catalyst/expressions/NamedArgumentExpression.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala index 6b4269f45f419..1e59e45f5b75f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.types.DataType * SQL Syntax: key => value * SQL grammar: key=identifier FAT_ARROW value=expression * - * Example usage in encode: + * Example usage with the "encode" scalar function: * SELECT encode("abc", charset => "utf-8"); * The second argument generates NamedArgumentExpression("charset", Literal("utf-8")) * SELECT encode(charset => "utf-8", value => "abc"); From b26fd433c10ae1864f5764d1ce983d8ca17f59b6 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Mon, 12 Jun 2023 16:23:20 -0700 Subject: [PATCH 14/36] Some changes --- .../sql/catalyst/parser/SqlBaseParser.g4 | 3 +- .../sql/catalyst/parser/AstBuilder.scala | 30 ++++++++++++------- .../apache/spark/sql/internal/SQLConf.scala | 7 +++++ 3 files changed, 29 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 0592400f980b7..c58cc91b93b7a 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -930,7 +930,8 @@ primaryExpression | LEFT_PAREN namedExpression (COMMA namedExpression)+ RIGHT_PAREN #rowConstructor | LEFT_PAREN query RIGHT_PAREN #subqueryExpression | IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN #identifierClause - | functionName LEFT_PAREN (setQuantifier? argument+=functionArgument (COMMA argument+=functionArgument)*)? RIGHT_PAREN + | functionName LEFT_PAREN (setQuantifier? argument+=functionArgument + (COMMA argument+=functionArgument)*)? RIGHT_PAREN (FILTER LEFT_PAREN WHERE where=booleanExpression RIGHT_PAREN)? (nullsOption=(IGNORE | RESPECT) NULLS)? ( OVER windowSpec)? #functionCall | identifier ARROW expression #lambda 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 d3ec4c1d9dd5d..e83b8592056d2 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 @@ -1545,11 +1545,16 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit throw QueryParsingErrors.invalidTableValuedFunctionNameError(name, ctx) } val args = func.functionArgument.asScala.map { e => - if (e.namedArgumentExpression != null) { - val key = e.namedArgumentExpression.key.getText - val value = e.namedArgumentExpression.value - NamedArgumentExpression(key, expression(value)) - } else { + Option(e.namedArgumentExpression).map { n => + if (SQLConf.get.getConf(SQLConf.ALLOW_NAMED_FUNCTION_ARGUMENTS)) { + NamedArgumentExpression(n.key.getText, expression(n.value)) + } else { + throw new ParseException( + errorClass = "Named arguments not enabled.", + messageParameters = Map("msg" -> (n.key.getText + " is a named argument.")), + ctx) + } + }.getOrElse { expression(e) } } @@ -2186,11 +2191,16 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val isDistinct = Option(ctx.setQuantifier()).exists(_.DISTINCT != null) // Call `toSeq`, otherwise `ctx.argument.asScala.map(expression)` is `Buffer` in Scala 2.13 val arguments = ctx.argument.asScala.map { e => - if (e.namedArgumentExpression != null) { - val key = e.namedArgumentExpression.key.getText - val value = e.namedArgumentExpression.value - NamedArgumentExpression(key, expression(value)) - } else { + Option(e.namedArgumentExpression).map { n => + if (SQLConf.get.getConf(SQLConf.ALLOW_NAMED_FUNCTION_ARGUMENTS)) { + NamedArgumentExpression(n.key.getText, expression(n.value)) + } else { + throw new ParseException( + errorClass = "Named arguments not enabled.", + messageParameters = Map("msg" -> (n.key.getText + " is a named argument.")), + ctx) + } + }.getOrElse { expression(e) } }.toSeq match { 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 47b8474953bb2..fe5346289a00b 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 @@ -321,6 +321,13 @@ object SQLConf { .booleanConf .createWithDefault(false) + val ALLOW_NAMED_FUNCTION_ARGUMENTS = buildConf("spark.sql.allowNamedFunctionArguments") + .doc("If true, Spark will turn on support for named parameters for all functions that has" + + " it implemented.") + .version("3.5.0") + .booleanConf + .createWithDefault(true) + val DYNAMIC_PARTITION_PRUNING_ENABLED = buildConf("spark.sql.optimizer.dynamicPartitionPruning.enabled") .doc("When true, we will generate predicate for partition column when it's used as join key") From 3ae0ee6956f2ce83c01b0a27e2e0255d37466c1a Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Mon, 12 Jun 2023 19:31:54 -0700 Subject: [PATCH 15/36] Adding test and error --- core/src/main/resources/error/error-classes.json | 5 +++++ .../spark/sql/catalyst/parser/AstBuilder.scala | 10 ++-------- .../spark/sql/errors/QueryCompilationErrors.scala | 7 +++++++ .../spark/sql/errors/QueryParsingErrorsSuite.scala | 13 ++++++++++++- 4 files changed, 26 insertions(+), 9 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index a12a800087072..f51026cdbe278 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1580,6 +1580,11 @@ "Not allowed to implement multiple UDF interfaces, UDF class ." ] }, + "NAMED_ARGUMENTS_SUPPORT_DISABLED" : { + "message" : [ + "Named arguments support is currently disabled. Enable suppport using ALLOW_NAMED_FUNCTION_ARGUMENTS." + ] + }, "NESTED_AGGREGATE_FUNCTION" : { "message" : [ "It is not allowed to use an aggregate function in the argument of another aggregate function. Please use the inner aggregate function in a sub-query." 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 e83b8592056d2..0c4d1dc97619d 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 @@ -1549,10 +1549,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit if (SQLConf.get.getConf(SQLConf.ALLOW_NAMED_FUNCTION_ARGUMENTS)) { NamedArgumentExpression(n.key.getText, expression(n.value)) } else { - throw new ParseException( - errorClass = "Named arguments not enabled.", - messageParameters = Map("msg" -> (n.key.getText + " is a named argument.")), - ctx) + throw QueryCompilationErrors.namedArgumentsNotEnabledError(n.key.getText) } }.getOrElse { expression(e) @@ -2195,10 +2192,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit if (SQLConf.get.getConf(SQLConf.ALLOW_NAMED_FUNCTION_ARGUMENTS)) { NamedArgumentExpression(n.key.getText, expression(n.value)) } else { - throw new ParseException( - errorClass = "Named arguments not enabled.", - messageParameters = Map("msg" -> (n.key.getText + " is a named argument.")), - ctx) + throw QueryCompilationErrors.namedArgumentsNotEnabledError(n.key.getText) } }.getOrElse { expression(e) 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 4c87b9da1c758..8fef87ca4a737 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 @@ -193,6 +193,13 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { messageParameters = Map("configKey" -> configKey)) } + def namedArgumentsNotEnabledError(argumentName: String): Throwable = { + new AnalysisException( + errorClass = "NAMED_ARGUMENTS_SUPPORT_DISABLED", + messageParameters = Map("message" -> (argumentName + " is a named argument.")) + ) + } + def unresolvedUsingColForJoinError( colName: String, suggestion: String, side: String): Throwable = { new AnalysisException( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala index 1be5f6469a5ad..4d3da6ca1c99a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala @@ -20,16 +20,27 @@ package org.apache.spark.sql.errors import org.apache.spark.SparkThrowable import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.test.SharedSparkSession // Turn of the length check because most of the tests check entire error messages // scalastyle:off line.size.limit -class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession { +class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQLHelper { private def parseException(sqlText: String): SparkThrowable = { intercept[ParseException](sql(sqlText).collect()) } + test("NAMED_ARGUMENTS_SUPPORT_DISABLED: named arguments not turned on") { + withSQLConf("spark.sql.allowNamedFunctionArguments" -> "false") { + checkError( + exception = parseException("SELECT * FROM encode(value => 'abc', charset => 'utf-8')"), + errorClass = "NAMED_ARGUMENTS_SUPPORT_DISABLED", + parameters = Map("message" -> "value is a named argument.") + ) + } + } + test("UNSUPPORTED_FEATURE: LATERAL join with NATURAL join not supported") { checkError( exception = parseException("SELECT * FROM t1 NATURAL JOIN LATERAL (SELECT c1 + c2 AS c2)"), From 4eff7196e1797764a83099c75452f8c879b5a5d7 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 13 Jun 2023 09:19:42 -0700 Subject: [PATCH 16/36] Making conversion --- .../scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 0c4d1dc97619d..219ef5d38b5bc 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 @@ -1554,7 +1554,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit }.getOrElse { expression(e) } - } + }.toSeq val tvf = UnresolvedTableValuedFunction(name, args) From 403877328961b4e3f8182e548ea38fdfaadce2fc Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 13 Jun 2023 16:42:11 -0700 Subject: [PATCH 17/36] Updating test to reflect error spec --- core/src/main/resources/error/error-classes.json | 2 +- .../org/apache/spark/sql/catalyst/parser/AstBuilder.scala | 6 ++++-- .../apache/spark/sql/errors/QueryCompilationErrors.scala | 4 ++-- .../apache/spark/sql/errors/QueryParsingErrorsSuite.scala | 2 +- 4 files changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index f51026cdbe278..27c8d9c1d3065 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1582,7 +1582,7 @@ }, "NAMED_ARGUMENTS_SUPPORT_DISABLED" : { "message" : [ - "Named arguments support is currently disabled. Enable suppport using ALLOW_NAMED_FUNCTION_ARGUMENTS." + "Cannot call function ‘’ because named argument references are not supported here. In this case, the named argument reference was ‘’.\n" ] }, "NESTED_AGGREGATE_FUNCTION" : { 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 219ef5d38b5bc..b4cd37bc1ed8d 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 @@ -1549,7 +1549,8 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit if (SQLConf.get.getConf(SQLConf.ALLOW_NAMED_FUNCTION_ARGUMENTS)) { NamedArgumentExpression(n.key.getText, expression(n.value)) } else { - throw QueryCompilationErrors.namedArgumentsNotEnabledError(n.key.getText) + throw QueryCompilationErrors.namedArgumentsNotEnabledError( + func.functionName.getText, n.key.getText) } }.getOrElse { expression(e) @@ -2192,7 +2193,8 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit if (SQLConf.get.getConf(SQLConf.ALLOW_NAMED_FUNCTION_ARGUMENTS)) { NamedArgumentExpression(n.key.getText, expression(n.value)) } else { - throw QueryCompilationErrors.namedArgumentsNotEnabledError(n.key.getText) + throw QueryCompilationErrors.namedArgumentsNotEnabledError( + name, n.key.getText) } }.getOrElse { expression(e) 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 8fef87ca4a737..f12098d725700 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 @@ -193,10 +193,10 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { messageParameters = Map("configKey" -> configKey)) } - def namedArgumentsNotEnabledError(argumentName: String): Throwable = { + def namedArgumentsNotEnabledError(functionName: String, argumentName: String): Throwable = { new AnalysisException( errorClass = "NAMED_ARGUMENTS_SUPPORT_DISABLED", - messageParameters = Map("message" -> (argumentName + " is a named argument.")) + messageParameters = Map("functionName" -> functionName, "argument" -> argumentName) ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala index 4d3da6ca1c99a..3ca59a1e1b97a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala @@ -36,7 +36,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL checkError( exception = parseException("SELECT * FROM encode(value => 'abc', charset => 'utf-8')"), errorClass = "NAMED_ARGUMENTS_SUPPORT_DISABLED", - parameters = Map("message" -> "value is a named argument.") + parameters = Map("functionName" -> "encode", "argument" -> "value") ) } } From 3209f6051ef17d254c04dcd6db2596e72ddb9b94 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Wed, 14 Jun 2023 10:03:09 -0700 Subject: [PATCH 18/36] Addressing some comments --- .../main/resources/error/error-classes.json | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 35 ++++++++----------- .../sql/errors/QueryCompilationErrors.scala | 3 +- .../sql/errors/QueryParsingErrorsSuite.scala | 8 ++++- 4 files changed, 25 insertions(+), 23 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 27c8d9c1d3065..8db648ef3ce0b 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1582,7 +1582,7 @@ }, "NAMED_ARGUMENTS_SUPPORT_DISABLED" : { "message" : [ - "Cannot call function ‘’ because named argument references are not supported here. In this case, the named argument reference was ‘’.\n" + "Cannot call function because named argument references are not supported here. In this case, the named argument reference was .\n" ] }, "NESTED_AGGREGATE_FUNCTION" : { 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 b4cd37bc1ed8d..35839b333eaa9 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 @@ -1513,6 +1513,19 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } } + private def extractExpression(expr: FunctionArgumentContext, funcName: String) : Expression = { + Option(expr.namedArgumentExpression).map { n => + if (SQLConf.get.getConf(SQLConf.ALLOW_NAMED_FUNCTION_ARGUMENTS)) { + NamedArgumentExpression(n.key.getText, expression(n.value)) + } else { + throw QueryCompilationErrors.namedArgumentsNotEnabledError( + funcName, n.key.getText) + } + }.getOrElse { + expression(expr) + } + } + private def withTimeTravel( ctx: TemporalClauseContext, plan: LogicalPlan): LogicalPlan = withOrigin(ctx) { val v = ctx.version @@ -1545,16 +1558,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit throw QueryParsingErrors.invalidTableValuedFunctionNameError(name, ctx) } val args = func.functionArgument.asScala.map { e => - Option(e.namedArgumentExpression).map { n => - if (SQLConf.get.getConf(SQLConf.ALLOW_NAMED_FUNCTION_ARGUMENTS)) { - NamedArgumentExpression(n.key.getText, expression(n.value)) - } else { - throw QueryCompilationErrors.namedArgumentsNotEnabledError( - func.functionName.getText, n.key.getText) - } - }.getOrElse { - expression(e) - } + extractExpression(e, func.functionName.getText) }.toSeq val tvf = UnresolvedTableValuedFunction(name, args) @@ -2189,16 +2193,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val isDistinct = Option(ctx.setQuantifier()).exists(_.DISTINCT != null) // Call `toSeq`, otherwise `ctx.argument.asScala.map(expression)` is `Buffer` in Scala 2.13 val arguments = ctx.argument.asScala.map { e => - Option(e.namedArgumentExpression).map { n => - if (SQLConf.get.getConf(SQLConf.ALLOW_NAMED_FUNCTION_ARGUMENTS)) { - NamedArgumentExpression(n.key.getText, expression(n.value)) - } else { - throw QueryCompilationErrors.namedArgumentsNotEnabledError( - name, n.key.getText) - } - }.getOrElse { - expression(e) - } + extractExpression(e, name) }.toSeq match { case Seq(UnresolvedStar(None)) if name.toLowerCase(Locale.ROOT) == "count" && !isDistinct => 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 f12098d725700..01264d1ce1f36 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 @@ -196,7 +196,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def namedArgumentsNotEnabledError(functionName: String, argumentName: String): Throwable = { new AnalysisException( errorClass = "NAMED_ARGUMENTS_SUPPORT_DISABLED", - messageParameters = Map("functionName" -> functionName, "argument" -> argumentName) + messageParameters = Map("functionName" -> toSQLId(functionName), + "argument" -> toSQLId(argumentName)) ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala index 3ca59a1e1b97a..a7d5046245df9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala @@ -21,6 +21,7 @@ import org.apache.spark.SparkThrowable import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLId import org.apache.spark.sql.test.SharedSparkSession // Turn of the length check because most of the tests check entire error messages @@ -36,7 +37,12 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL checkError( exception = parseException("SELECT * FROM encode(value => 'abc', charset => 'utf-8')"), errorClass = "NAMED_ARGUMENTS_SUPPORT_DISABLED", - parameters = Map("functionName" -> "encode", "argument" -> "value") + parameters = Map("functionName" -> toSQLId("encode"), "argument" -> toSQLId("value")) + ) + checkError( + exception = parseException("SELECT explode(arr => array(10, 20))"), + errorClass = "NAMED_ARGUMENTS_SUPPORT_DISABLED", + parameters = Map("functionName"-> toSQLId("explode"), "argument" -> toSQLId("arr")) ) } } From 23075d511c765fdefaebe342da97d2c1a9eb7719 Mon Sep 17 00:00:00 2001 From: Richard Yu <134337791+learningchess2003@users.noreply.github.com> Date: Tue, 20 Jun 2023 15:59:35 -0700 Subject: [PATCH 19/36] Update sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala Co-authored-by: Maxim Gekk --- .../sql/catalyst/expressions/NamedArgumentExpression.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala index 1e59e45f5b75f..79480a7ea6b95 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala @@ -41,7 +41,7 @@ import org.apache.spark.sql.types.DataType * @param value The value of the function argument */ case class NamedArgumentExpression(key: String, value: Expression) - extends UnaryExpression with Unevaluable { + extends UnaryExpression with Unevaluable { override def nullable: Boolean = value.nullable override def dataType: DataType = value.dataType From c87901208e6bb567038dfc892997c94fc40b4cc6 Mon Sep 17 00:00:00 2001 From: Richard Yu <134337791+learningchess2003@users.noreply.github.com> Date: Tue, 20 Jun 2023 15:59:41 -0700 Subject: [PATCH 20/36] Update sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala Co-authored-by: Maxim Gekk --- .../scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 35839b333eaa9..82c47437d925e 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 @@ -1515,7 +1515,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit private def extractExpression(expr: FunctionArgumentContext, funcName: String) : Expression = { Option(expr.namedArgumentExpression).map { n => - if (SQLConf.get.getConf(SQLConf.ALLOW_NAMED_FUNCTION_ARGUMENTS)) { + if (conf.getConf(SQLConf.ALLOW_NAMED_FUNCTION_ARGUMENTS)) { NamedArgumentExpression(n.key.getText, expression(n.value)) } else { throw QueryCompilationErrors.namedArgumentsNotEnabledError( From c60f1a4a1746d09e5b4e6f5e5d51728cce7c3717 Mon Sep 17 00:00:00 2001 From: Richard Yu <134337791+learningchess2003@users.noreply.github.com> Date: Tue, 20 Jun 2023 15:59:48 -0700 Subject: [PATCH 21/36] Update sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala Co-authored-by: Maxim Gekk --- .../spark/sql/catalyst/parser/ExpressionParserSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 36ea89e2fdd75..adc82312fbb53 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -332,8 +332,8 @@ class ExpressionParserSuite extends AnalysisTest { test("function expressions with named arguments") { assertEqual("encode(value => 'abc', charset => 'utf-8')", - $"encode".function(NamedArgumentExpression("value", Literal("abc")), - NamedArgumentExpression("charset", Literal("utf-8")))) + $"encode".function(NamedArgumentExpression("value", Literal("abc")), + NamedArgumentExpression("charset", Literal("utf-8")))) assertEqual("encode('abc', charset => 'utf-8')", $"encode".function(Literal("abc"), NamedArgumentExpression("charset", Literal("utf-8")))) From 208d0058ac1ea1e95e8767a6295d0ad40d428fb4 Mon Sep 17 00:00:00 2001 From: Richard Yu <134337791+learningchess2003@users.noreply.github.com> Date: Tue, 20 Jun 2023 15:59:56 -0700 Subject: [PATCH 22/36] Update sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala Co-authored-by: Maxim Gekk --- .../org/apache/spark/sql/errors/QueryCompilationErrors.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 01264d1ce1f36..3f394da0e3fe6 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 @@ -196,7 +196,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def namedArgumentsNotEnabledError(functionName: String, argumentName: String): Throwable = { new AnalysisException( errorClass = "NAMED_ARGUMENTS_SUPPORT_DISABLED", - messageParameters = Map("functionName" -> toSQLId(functionName), + messageParameters = Map( + "functionName" -> toSQLId(functionName), "argument" -> toSQLId(argumentName)) ) } From 90dafcbd27e036aff27c89af0f0d7087c96cad53 Mon Sep 17 00:00:00 2001 From: Richard Yu <134337791+learningchess2003@users.noreply.github.com> Date: Tue, 20 Jun 2023 16:00:02 -0700 Subject: [PATCH 23/36] Update sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala Co-authored-by: Maxim Gekk --- .../org/apache/spark/sql/catalyst/parser/AstBuilder.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 82c47437d925e..7a75840d18b1b 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 @@ -1518,8 +1518,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit if (conf.getConf(SQLConf.ALLOW_NAMED_FUNCTION_ARGUMENTS)) { NamedArgumentExpression(n.key.getText, expression(n.value)) } else { - throw QueryCompilationErrors.namedArgumentsNotEnabledError( - funcName, n.key.getText) + throw QueryCompilationErrors.namedArgumentsNotEnabledError(funcName, n.key.getText) } }.getOrElse { expression(expr) From dc868e7b1c682c622cdcd05e7a4ff26a0108d536 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 20 Jun 2023 16:43:08 -0700 Subject: [PATCH 24/36] Adding addressed comments --- core/src/main/resources/error/error-classes.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 8db648ef3ce0b..a2b9ac5b87708 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1582,7 +1582,7 @@ }, "NAMED_ARGUMENTS_SUPPORT_DISABLED" : { "message" : [ - "Cannot call function because named argument references are not supported here. In this case, the named argument reference was .\n" + "Cannot call function because named argument references are not enabled here. In this case, the named argument reference was . Set spark.sql.allowNamedFunctionArguments to true to turn on feature." ] }, "NESTED_AGGREGATE_FUNCTION" : { From 9a5f74e18c4bf4544fc087d0406eb41a56680d5f Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Thu, 22 Jun 2023 12:43:21 -0700 Subject: [PATCH 25/36] Fixing scalastyle rebase error --- .../spark/sql/catalyst/parser/AstBuilder.scala | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) 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 5b7343948de31..bf82cba769647 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 @@ -1563,18 +1563,17 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit Seq.empty } - withIdentClause( - func.functionName.expression, - () => getFunctionMultiparts(func.functionName), - name => { - if (name.length > 1) { - throw QueryParsingErrors.invalidTableValuedFunctionNameError(name, ctx) + withFuncIdentClause( + func.functionName, + ident => { + if (ident.length > 1) { + throw QueryParsingErrors.invalidTableValuedFunctionNameError(ident, ctx) } val args = func.functionArgument.asScala.map { e => extractExpression(e, func.functionName.getText) }.toSeq - val tvf = UnresolvedTableValuedFunction(name, args) + val tvf = UnresolvedTableValuedFunction(ident, args) val tvfAliases = if (aliases.nonEmpty) UnresolvedTVFAliases(ident, tvf, aliases) else tvf From 566c9946bea2765d12c2db16d48faa9df1770a0a Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Thu, 22 Jun 2023 12:48:23 -0700 Subject: [PATCH 26/36] Fixing some problems --- .../org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index 988132260be26..0d250cce5b847 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -31,7 +31,6 @@ import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, R import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{NamedParameter, UnresolvedGenerator} import org.apache.spark.sql.catalyst.expressions.{Grouping, Literal, NamedArgumentExpression, RowNumber} - import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.expressions.objects.InitializeJavaBean From 65d9d95ab04ca26bcf5fe3a2623b9cd3404ebb6a Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 27 Jun 2023 09:59:09 -0700 Subject: [PATCH 27/36] Fixing indentation --- .../catalyst/parser/ExpressionParserSuite.scala | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 751edc2c3cada..1b9c2709ecd1f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -335,20 +335,16 @@ class ExpressionParserSuite extends AnalysisTest { $"encode".function(NamedArgumentExpression("value", Literal("abc")), NamedArgumentExpression("charset", Literal("utf-8")))) assertEqual("encode('abc', charset => 'utf-8')", - $"encode".function(Literal("abc"), - NamedArgumentExpression("charset", Literal("utf-8")))) + $"encode".function(Literal("abc"), NamedArgumentExpression("charset", Literal("utf-8")))) assertEqual("encode(charset => 'utf-8', 'abc')", - $"encode".function( - NamedArgumentExpression("charset", Literal("utf-8")), Literal("abc"))) + $"encode".function(NamedArgumentExpression("charset", Literal("utf-8")), Literal("abc"))) assertEqual("encode('abc', charset => 'utf' || '-8')", - $"encode".function(Literal("abc"), - NamedArgumentExpression("charset", - Concat(Literal("utf") :: Literal("-8") :: Nil)))) + $"encode".function(Literal("abc"), NamedArgumentExpression("charset", + Concat(Literal("utf") :: Literal("-8") :: Nil)))) val unresolvedAlias = Project(Seq(UnresolvedAlias(Literal("1"))), OneRowRelation()) assertEqual("encode(value => ((select '1')), charset => 'utf-8')", - $"encode".function( - NamedArgumentExpression("value", ScalarSubquery(plan = unresolvedAlias)), - NamedArgumentExpression("charset", Literal("utf-8")))) + $"encode".function(NamedArgumentExpression("value", ScalarSubquery(plan = unresolvedAlias)), + NamedArgumentExpression("charset", Literal("utf-8")))) } private def lv(s: Symbol) = UnresolvedNamedLambdaVariable(Seq(s.name)) From 765322556aef934a6f42f90983c67a6dbd34a416 Mon Sep 17 00:00:00 2001 From: Richard Yu <134337791+learningchess2003@users.noreply.github.com> Date: Tue, 27 Jun 2023 10:04:10 -0700 Subject: [PATCH 28/36] Update core/src/main/resources/error/error-classes.json Co-authored-by: Maxim Gekk --- core/src/main/resources/error/error-classes.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 5e0265ff3e54a..20b9682b9c681 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1661,7 +1661,7 @@ }, "NAMED_ARGUMENTS_SUPPORT_DISABLED" : { "message" : [ - "Cannot call function because named argument references are not enabled here. In this case, the named argument reference was . Set spark.sql.allowNamedFunctionArguments to true to turn on feature." + "Cannot call function because named argument references are not enabled here. In this case, the named argument reference was . Set \"spark.sql.allowNamedFunctionArguments\" to \"true\" to turn on feature." ] }, "NESTED_AGGREGATE_FUNCTION" : { From f071073e486fe1de1b94bb436d32e274ad9604cf Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 27 Jun 2023 14:48:49 -0700 Subject: [PATCH 29/36] Addressing comments --- .../expressions/NamedArgumentExpression.scala | 1 - .../spark/sql/catalyst/parser/AstBuilder.scala | 6 +++--- .../spark/sql/errors/QueryExecutionErrorsSuite.scala | 12 ------------ 3 files changed, 3 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala index 79480a7ea6b95..e8e6980805bda 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/NamedArgumentExpression.scala @@ -42,7 +42,6 @@ import org.apache.spark.sql.types.DataType */ case class NamedArgumentExpression(key: String, value: Expression) extends UnaryExpression with Unevaluable { - override def nullable: Boolean = value.nullable override def dataType: DataType = value.dataType 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 bf82cba769647..9a395924c451c 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 @@ -1527,7 +1527,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit } } - private def extractExpression(expr: FunctionArgumentContext, funcName: String) : Expression = { + private def extractNamedArgument(expr: FunctionArgumentContext, funcName: String) : Expression = { Option(expr.namedArgumentExpression).map { n => if (conf.getConf(SQLConf.ALLOW_NAMED_FUNCTION_ARGUMENTS)) { NamedArgumentExpression(n.key.getText, expression(n.value)) @@ -1570,7 +1570,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit throw QueryParsingErrors.invalidTableValuedFunctionNameError(ident, ctx) } val args = func.functionArgument.asScala.map { e => - extractExpression(e, func.functionName.getText) + extractNamedArgument(e, func.functionName.getText) }.toSeq val tvf = UnresolvedTableValuedFunction(ident, args) @@ -2204,7 +2204,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val isDistinct = Option(ctx.setQuantifier()).exists(_.DISTINCT != null) // Call `toSeq`, otherwise `ctx.argument.asScala.map(expression)` is `Buffer` in Scala 2.13 val arguments = ctx.argument.asScala.map { e => - extractExpression(e, name) + extractNamedArgument(e, name) }.toSeq match { case Seq(UnresolvedStar(None)) if name.toLowerCase(Locale.ROOT) == "count" && !isDistinct => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index 0d250cce5b847..e1076f4c8f075 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -875,18 +875,6 @@ class QueryExecutionErrorsSuite sqlState = "XX000") } - test("INTERNAL_ERROR: Calling eval on Unevaluable NamedArgumentExpression") { - val e = intercept[SparkException] { - NamedArgumentExpression("arg0", Literal("value0")).eval() - } - checkError( - exception = e, - errorClass = "INTERNAL_ERROR", - parameters = Map("message" -> "Cannot evaluate expression: arg0 => value0"), - sqlState = "XX000" - ) - } - test("INTERNAL_ERROR: Calling doGenCode on unresolved") { val e = intercept[SparkException] { val ctx = new CodegenContext From 7d99b58d1570f9f7e5a3297392ddf658339b9916 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Wed, 28 Jun 2023 15:12:59 -0700 Subject: [PATCH 30/36] Add golden file for end-to-end tests --- .../named-function-arguments.sql.out | 133 ++++++++++++++++ .../inputs/named-function-arguments.sql | 6 + .../results/named-function-arguments.sql.out | 145 ++++++++++++++++++ .../errors/QueryExecutionErrorsSuite.scala | 2 +- 4 files changed, 285 insertions(+), 1 deletion(-) create mode 100644 sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/named-function-arguments.sql.out diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out new file mode 100644 index 0000000000000..a7e7a8d0a5dcf --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out @@ -0,0 +1,133 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT mask('abcd-EFGH-8765-4321', 'A', 'abc') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "lowerChar", + "sqlExpr" : "\"mask(abcd-EFGH-8765-4321, A, abc, n, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "mask('abcd-EFGH-8765-4321', 'A', 'abc')" + } ] +} + + +-- !query +SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"namedargumentexpression(q)\"", + "inputName" : "upperChar", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"mask(AbCD123-@$#, namedargumentexpression(q), namedargumentexpression(Q), namedargumentexpression(o), namedargumentexpression(d))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 98, + "fragment" : "mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd')" + } ] +} + + +-- !query +SELECT mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbCD123-@$#') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"namedargumentexpression(Q)\"", + "inputName" : "upperChar", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"mask(namedargumentexpression(q), namedargumentexpression(Q), namedargumentexpression(o), namedargumentexpression(d), namedargumentexpression(AbCD123-@$#))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 105, + "fragment" : "mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbCD123-@$#')" + } ] +} + + +-- !query +SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', digitChar => 'd') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"namedargumentexpression(q)\"", + "inputName" : "upperChar", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"mask(AbCD123-@$#, namedargumentexpression(q), namedargumentexpression(Q), namedargumentexpression(d), NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', digitChar => 'd')" + } ] +} + + +-- !query +SELECT mask(lowerChar => 'q', upperChar => 'Q', digitChar => 'd', str => 'AbCD123-@$#') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"namedargumentexpression(Q)\"", + "inputName" : "upperChar", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"mask(namedargumentexpression(q), namedargumentexpression(Q), namedargumentexpression(d), namedargumentexpression(AbCD123-@$#), NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 87, + "fragment" : "mask(lowerChar => 'q', upperChar => 'Q', digitChar => 'd', str => 'AbCD123-@$#')" + } ] +} + + +-- !query +SELECT mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "upperChar", + "sqlExpr" : "\"mask(namedargumentexpression(q), AbCD123-@$#, namedargumentexpression(Q), namedargumentexpression(o), namedargumentexpression(d))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 98, + "fragment" : "mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd')" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql b/sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql new file mode 100644 index 0000000000000..fc0c48d5723a4 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql @@ -0,0 +1,6 @@ +SELECT mask('abcd-EFGH-8765-4321', 'A', 'abc'); +SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd'); +SELECT mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbCD123-@$#'); +SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', digitChar => 'd'); +SELECT mask(lowerChar => 'q', upperChar => 'Q', digitChar => 'd', str => 'AbCD123-@$#'); +SELECT mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd'); diff --git a/sql/core/src/test/resources/sql-tests/results/named-function-arguments.sql.out b/sql/core/src/test/resources/sql-tests/results/named-function-arguments.sql.out new file mode 100644 index 0000000000000..12396ae9a8910 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/named-function-arguments.sql.out @@ -0,0 +1,145 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT mask('abcd-EFGH-8765-4321', 'A', 'abc') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "lowerChar", + "sqlExpr" : "\"mask(abcd-EFGH-8765-4321, A, abc, n, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "mask('abcd-EFGH-8765-4321', 'A', 'abc')" + } ] +} + + +-- !query +SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"namedargumentexpression(q)\"", + "inputName" : "upperChar", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"mask(AbCD123-@$#, namedargumentexpression(q), namedargumentexpression(Q), namedargumentexpression(o), namedargumentexpression(d))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 98, + "fragment" : "mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd')" + } ] +} + + +-- !query +SELECT mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbCD123-@$#') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"namedargumentexpression(Q)\"", + "inputName" : "upperChar", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"mask(namedargumentexpression(q), namedargumentexpression(Q), namedargumentexpression(o), namedargumentexpression(d), namedargumentexpression(AbCD123-@$#))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 105, + "fragment" : "mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbCD123-@$#')" + } ] +} + + +-- !query +SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', digitChar => 'd') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"namedargumentexpression(q)\"", + "inputName" : "upperChar", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"mask(AbCD123-@$#, namedargumentexpression(q), namedargumentexpression(Q), namedargumentexpression(d), NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', digitChar => 'd')" + } ] +} + + +-- !query +SELECT mask(lowerChar => 'q', upperChar => 'Q', digitChar => 'd', str => 'AbCD123-@$#') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"namedargumentexpression(Q)\"", + "inputName" : "upperChar", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"mask(namedargumentexpression(q), namedargumentexpression(Q), namedargumentexpression(d), namedargumentexpression(AbCD123-@$#), NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 87, + "fragment" : "mask(lowerChar => 'q', upperChar => 'Q', digitChar => 'd', str => 'AbCD123-@$#')" + } ] +} + + +-- !query +SELECT mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "upperChar", + "sqlExpr" : "\"mask(namedargumentexpression(q), AbCD123-@$#, namedargumentexpression(Q), namedargumentexpression(o), namedargumentexpression(d))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 98, + "fragment" : "mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd')" + } ] +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index e1076f4c8f075..8f47b06d8552e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark._ import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{NamedParameter, UnresolvedGenerator} -import org.apache.spark.sql.catalyst.expressions.{Grouping, Literal, NamedArgumentExpression, RowNumber} +import org.apache.spark.sql.catalyst.expressions.{Grouping, Literal, RowNumber} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.expressions.objects.InitializeJavaBean From 345b96d790e1ce86c9cc202522bd98019125ecbd Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Wed, 28 Jun 2023 15:18:13 -0700 Subject: [PATCH 31/36] Remove unnecessary line --- .../named-function-arguments.sql.out | 21 ----------------- .../inputs/named-function-arguments.sql | 1 - .../results/named-function-arguments.sql.out | 23 ------------------- 3 files changed, 45 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out index a7e7a8d0a5dcf..faa05535cb322 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out @@ -1,25 +1,4 @@ -- Automatically generated by SQLQueryTestSuite --- !query -SELECT mask('abcd-EFGH-8765-4321', 'A', 'abc') --- !query analysis -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE", - "sqlState" : "42K09", - "messageParameters" : { - "exprName" : "lowerChar", - "sqlExpr" : "\"mask(abcd-EFGH-8765-4321, A, abc, n, NULL)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 46, - "fragment" : "mask('abcd-EFGH-8765-4321', 'A', 'abc')" - } ] -} - - -- !query SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd') -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql b/sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql index fc0c48d5723a4..aeb7b1e85cd8c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql @@ -1,4 +1,3 @@ -SELECT mask('abcd-EFGH-8765-4321', 'A', 'abc'); SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd'); SELECT mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbCD123-@$#'); SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', digitChar => 'd'); diff --git a/sql/core/src/test/resources/sql-tests/results/named-function-arguments.sql.out b/sql/core/src/test/resources/sql-tests/results/named-function-arguments.sql.out index 12396ae9a8910..842374542ec6e 100644 --- a/sql/core/src/test/resources/sql-tests/results/named-function-arguments.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/named-function-arguments.sql.out @@ -1,27 +1,4 @@ -- Automatically generated by SQLQueryTestSuite --- !query -SELECT mask('abcd-EFGH-8765-4321', 'A', 'abc') --- !query schema -struct<> --- !query output -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE", - "sqlState" : "42K09", - "messageParameters" : { - "exprName" : "lowerChar", - "sqlExpr" : "\"mask(abcd-EFGH-8765-4321, A, abc, n, NULL)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 46, - "fragment" : "mask('abcd-EFGH-8765-4321', 'A', 'abc')" - } ] -} - - -- !query SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd') -- !query schema From 704dcad2fd93b1e2fb244a43eac8a3e7f701eecd Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Thu, 29 Jun 2023 10:52:34 -0700 Subject: [PATCH 32/36] Line deletion --- .../org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index c7e0b115d5899..228a287e14f49 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -1435,7 +1435,6 @@ class PlanParserSuite extends AnalysisTest { NamedArgumentExpression("arg1", Literal("value1")) :: Literal(2) :: NamedArgumentExpression("arg2", Literal(true)) :: Nil).select(star())) - assertEqual( "select * from my_tvf(group => 'abc')", UnresolvedTableValuedFunction("my_tvf", From 01c3af98a80042830a15d85d7ce2212c7a34f867 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Thu, 29 Jun 2023 13:09:58 -0700 Subject: [PATCH 33/36] Reverting error class change --- core/src/main/resources/error/error-classes.json | 5 ----- 1 file changed, 5 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 20b9682b9c681..264d9b7c3a033 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1659,11 +1659,6 @@ "Not allowed to implement multiple UDF interfaces, UDF class ." ] }, - "NAMED_ARGUMENTS_SUPPORT_DISABLED" : { - "message" : [ - "Cannot call function because named argument references are not enabled here. In this case, the named argument reference was . Set \"spark.sql.allowNamedFunctionArguments\" to \"true\" to turn on feature." - ] - }, "NESTED_AGGREGATE_FUNCTION" : { "message" : [ "It is not allowed to use an aggregate function in the argument of another aggregate function. Please use the inner aggregate function in a sub-query." From c6ca3bd6f16ad8d9f6284f10500e423d8159ae20 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Thu, 29 Jun 2023 13:11:24 -0700 Subject: [PATCH 34/36] Try --- core/src/main/resources/error/error-classes.json | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 264d9b7c3a033..20b9682b9c681 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1659,6 +1659,11 @@ "Not allowed to implement multiple UDF interfaces, UDF class ." ] }, + "NAMED_ARGUMENTS_SUPPORT_DISABLED" : { + "message" : [ + "Cannot call function because named argument references are not enabled here. In this case, the named argument reference was . Set \"spark.sql.allowNamedFunctionArguments\" to \"true\" to turn on feature." + ] + }, "NESTED_AGGREGATE_FUNCTION" : { "message" : [ "It is not allowed to use an aggregate function in the argument of another aggregate function. Please use the inner aggregate function in a sub-query." From da49eedabd23b2e46c737445f7b917e2cc8badbc Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Thu, 29 Jun 2023 13:12:36 -0700 Subject: [PATCH 35/36] Moving stuff around --- core/src/main/resources/error/error-classes.json | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 20b9682b9c681..7c0e52941797d 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1,4 +1,9 @@ { + "NAMED_ARGUMENTS_SUPPORT_DISABLED" : { + "message" : [ + "Cannot call function because named argument references are not enabled here. In this case, the named argument reference was . Set \"spark.sql.allowNamedFunctionArguments\" to \"true\" to turn on feature." + ] + }, "AGGREGATE_FUNCTION_WITH_NONDETERMINISTIC_EXPRESSION" : { "message" : [ "Non-deterministic expression should not appear in the arguments of an aggregate function." @@ -1659,11 +1664,6 @@ "Not allowed to implement multiple UDF interfaces, UDF class ." ] }, - "NAMED_ARGUMENTS_SUPPORT_DISABLED" : { - "message" : [ - "Cannot call function because named argument references are not enabled here. In this case, the named argument reference was . Set \"spark.sql.allowNamedFunctionArguments\" to \"true\" to turn on feature." - ] - }, "NESTED_AGGREGATE_FUNCTION" : { "message" : [ "It is not allowed to use an aggregate function in the argument of another aggregate function. Please use the inner aggregate function in a sub-query." From 57357324a8cc4213ecae447c9c517df26e86e399 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Thu, 29 Jun 2023 13:13:33 -0700 Subject: [PATCH 36/36] Original --- core/src/main/resources/error/error-classes.json | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 7c0e52941797d..20b9682b9c681 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1,9 +1,4 @@ { - "NAMED_ARGUMENTS_SUPPORT_DISABLED" : { - "message" : [ - "Cannot call function because named argument references are not enabled here. In this case, the named argument reference was . Set \"spark.sql.allowNamedFunctionArguments\" to \"true\" to turn on feature." - ] - }, "AGGREGATE_FUNCTION_WITH_NONDETERMINISTIC_EXPRESSION" : { "message" : [ "Non-deterministic expression should not appear in the arguments of an aggregate function." @@ -1664,6 +1659,11 @@ "Not allowed to implement multiple UDF interfaces, UDF class ." ] }, + "NAMED_ARGUMENTS_SUPPORT_DISABLED" : { + "message" : [ + "Cannot call function because named argument references are not enabled here. In this case, the named argument reference was . Set \"spark.sql.allowNamedFunctionArguments\" to \"true\" to turn on feature." + ] + }, "NESTED_AGGREGATE_FUNCTION" : { "message" : [ "It is not allowed to use an aggregate function in the argument of another aggregate function. Please use the inner aggregate function in a sub-query."