-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-44059] Add analyzer support of named arguments for built-in functions #41864
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
7b9d696
c905b1e
be77e54
d88129d
1c6717d
803a005
60a61c2
9728ce3
1f55065
f4fc8d1
67440d2
cc82b0e
a9cb890
3706e34
1feb222
61fb514
a2709c8
6eb3080
46d258f
5bb1d0c
887079d
db13c24
45c6a4e
53d1659
d30fa3d
03efd52
5bd65f0
6914a2b
9317f2f
4660881
60d1f06
8ac8e4c
b6c3d8a
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||||
|---|---|---|---|---|---|---|
|
|
@@ -733,6 +733,24 @@ | |||||
| ], | ||||||
| "sqlState" : "23505" | ||||||
| }, | ||||||
| "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT" : { | ||||||
| "message" : [ | ||||||
| "Call to function <functionName> is invalid because it includes multiple argument assignments to the same parameter name <parameterName>." | ||||||
| ], | ||||||
| "subClass" : { | ||||||
| "BOTH_POSITIONAL_AND_NAMED" : { | ||||||
| "message" : [ | ||||||
| "A positional argument and named argument both referred to the same parameter." | ||||||
| ] | ||||||
| }, | ||||||
| "DOUBLE_NAMED_ARGUMENT_REFERENCE" : { | ||||||
| "message" : [ | ||||||
| "More than one named argument referred to the same parameter." | ||||||
| ] | ||||||
| } | ||||||
| }, | ||||||
| "sqlState" : "4274K" | ||||||
| }, | ||||||
| "EMPTY_JSON_FIELD_VALUE" : { | ||||||
| "message" : [ | ||||||
| "Failed to parse an empty string for data type <dataType>." | ||||||
|
|
@@ -1893,7 +1911,13 @@ | |||||
| "Not allowed to implement multiple UDF interfaces, UDF class <className>." | ||||||
| ] | ||||||
| }, | ||||||
| "NAMED_ARGUMENTS_SUPPORT_DISABLED" : { | ||||||
| "NAMED_PARAMETERS_NOT_SUPPORTED" : { | ||||||
| "message" : [ | ||||||
| "Named parameters are not supported for function <functionName>; please retry the query with positional arguments to the function call instead." | ||||||
| ], | ||||||
| "sqlState" : "4274K" | ||||||
| }, | ||||||
| "NAMED_PARAMETER_SUPPORT_DISABLED" : { | ||||||
| "message" : [ | ||||||
| "Cannot call function <functionName> because named argument references are not enabled here. In this case, the named argument reference was <argument>. Set \"spark.sql.allowNamedFunctionArguments\" to \"true\" to turn on feature." | ||||||
| ] | ||||||
|
|
@@ -2226,6 +2250,12 @@ | |||||
| ], | ||||||
| "sqlState" : "42614" | ||||||
| }, | ||||||
| "REQUIRED_PARAMETER_NOT_FOUND" : { | ||||||
| "message" : [ | ||||||
| "Cannot invoke function <functionName> because the parameter named <parameterName> is required, but the function call did not supply a value. Please update the function call to supply an argument value (either positionally or by name) and retry the query again." | ||||||
| ], | ||||||
| "sqlState" : "4274K" | ||||||
| }, | ||||||
| "REQUIRES_SINGLE_PART_NAMESPACE" : { | ||||||
| "message" : [ | ||||||
| "<sessionCatalog> requires a single-part namespace, but got <namespace>." | ||||||
|
|
@@ -2416,6 +2446,12 @@ | |||||
| ], | ||||||
| "sqlState" : "42K09" | ||||||
| }, | ||||||
| "UNEXPECTED_POSITIONAL_ARGUMENT" : { | ||||||
| "message" : [ | ||||||
| "Cannot invoke function <functionName> because it contains positional argument(s) following named argument(s); please rearrange them so the positional arguments come first and then retry the query again." | ||||||
| ], | ||||||
| "sqlState" : "4274K" | ||||||
| }, | ||||||
| "UNKNOWN_PROTOBUF_MESSAGE_TYPE" : { | ||||||
| "message" : [ | ||||||
| "Attempting to treat <descriptorName> as a Message, but it was <containingType>." | ||||||
|
|
@@ -2445,6 +2481,12 @@ | |||||
| ], | ||||||
| "sqlState" : "428C4" | ||||||
| }, | ||||||
| "UNRECOGNIZED_PARAMETER_NAME" : { | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I asked Serge, and the original name is right. There's probably some confusion over the technical difference between argument and parameter. Typically, parameter is what each name is referring to. |
||||||
| "message" : [ | ||||||
| "Cannot invoke function <functionName> because the function call included a named argument reference for the argument named <argumentName>, but this function does not include any signature containing an argument with this name. Did you mean one of the following? [<proposal>]." | ||||||
| ], | ||||||
| "sqlState" : "4274K" | ||||||
| }, | ||||||
| "UNRECOGNIZED_SQL_TYPE" : { | ||||||
| "message" : [ | ||||||
| "Unrecognized SQL type - name: <typeName>, id: <jdbcType>." | ||||||
|
|
||||||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,36 @@ | ||
| --- | ||
| layout: global | ||
| title: DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT error class | ||
| displayTitle: DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT error class | ||
| license: | | ||
| 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. | ||
| --- | ||
|
|
||
| [SQLSTATE: 4274K](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) | ||
|
|
||
| Call to function `<functionName>` is invalid because it includes multiple argument assignments to the same parameter name `<parameterName>`. | ||
|
|
||
| This error class has the following derived error classes: | ||
|
|
||
| ## BOTH_POSITIONAL_AND_NAMED | ||
|
|
||
| A positional argument and named argument both referred to the same parameter. | ||
|
|
||
| ## DOUBLE_NAMED_ARGUMENT_REFERENCE | ||
|
|
||
| More than one named argument referred to the same parameter. | ||
|
|
||
|
|
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.FunctionIdentifier | |
| import org.apache.spark.sql.catalyst.expressions._ | ||
| import org.apache.spark.sql.catalyst.expressions.aggregate._ | ||
| import org.apache.spark.sql.catalyst.expressions.xml._ | ||
| import org.apache.spark.sql.catalyst.plans.logical.{Generate, LogicalPlan, OneRowRelation, Range} | ||
| import org.apache.spark.sql.catalyst.plans.logical.{FunctionBuilderBase, Generate, LogicalPlan, OneRowRelation, Range} | ||
| import org.apache.spark.sql.catalyst.trees.TreeNodeTag | ||
| import org.apache.spark.sql.errors.QueryCompilationErrors | ||
| import org.apache.spark.sql.types._ | ||
|
|
@@ -358,8 +358,8 @@ object FunctionRegistry { | |
| // misc non-aggregate functions | ||
| expression[Abs]("abs"), | ||
| expression[Coalesce]("coalesce"), | ||
| expression[Explode]("explode"), | ||
| expressionGeneratorOuter[Explode]("explode_outer"), | ||
| expressionBuilder("explode", ExplodeExpressionBuilder), | ||
| expressionGeneratorBuilderOuter("explode_outer", ExplodeExpressionBuilder), | ||
| expression[Greatest]("greatest"), | ||
| expression[If]("if"), | ||
| expression[Inline]("inline"), | ||
|
|
@@ -491,7 +491,7 @@ object FunctionRegistry { | |
| expression[CollectList]("collect_list"), | ||
| expression[CollectList]("array_agg", true, Some("3.3.0")), | ||
| expression[CollectSet]("collect_set"), | ||
| expression[CountMinSketchAgg]("count_min_sketch"), | ||
| expressionBuilder("count_min_sketch", CountMinSketchAggExpressionBuilder), | ||
| expression[BoolAnd]("every", true), | ||
| expression[BoolAnd]("bool_and"), | ||
| expression[BoolOr]("any", true), | ||
|
|
@@ -823,7 +823,7 @@ object FunctionRegistry { | |
| castAlias("string", StringType), | ||
|
|
||
| // mask functions | ||
| expression[Mask]("mask"), | ||
| expressionBuilder("mask", MaskExpressionBuilder), | ||
|
|
||
| // csv | ||
| expression[CsvToStructs]("from_csv"), | ||
|
|
@@ -887,13 +887,42 @@ object FunctionRegistry { | |
| since: Option[String] = None): (String, (ExpressionInfo, FunctionBuilder)) = { | ||
| val (expressionInfo, builder) = FunctionRegistryBase.build[T](name, since) | ||
| val newBuilder = (expressions: Seq[Expression]) => { | ||
| if (expressions.exists(_.isInstanceOf[NamedArgumentExpression])) { | ||
| throw QueryCompilationErrors.namedArgumentsNotSupported(name) | ||
| } | ||
| val expr = builder(expressions) | ||
| if (setAlias) expr.setTagValue(FUNC_ALIAS, name) | ||
| expr | ||
| } | ||
| (name, (expressionInfo, newBuilder)) | ||
| } | ||
|
|
||
| /** | ||
| * This method will be used to rearrange the arguments provided in function invocation | ||
| * in the order defined by the function signature given in the builder instance. | ||
| * | ||
| * @param name The name of the function | ||
| * @param builder The builder of the function expression | ||
| * @param expressions The argument list passed in function invocation | ||
| * @tparam T The class of the builder | ||
| * @return An argument list in positional order defined by the builder | ||
| */ | ||
| def rearrangeExpressions[T <: FunctionBuilderBase[_]]( | ||
| name: String, | ||
| builder: T, | ||
| expressions: Seq[Expression]) : Seq[Expression] = { | ||
| val rearrangedExpressions = if (!builder.functionSignature.isEmpty) { | ||
| val functionSignature = builder.functionSignature.get | ||
| builder.rearrange(functionSignature, expressions, name) | ||
| } else { | ||
| expressions | ||
| } | ||
| if (rearrangedExpressions.exists(_.isInstanceOf[NamedArgumentExpression])) { | ||
| throw QueryCompilationErrors.namedArgumentsNotSupported(name) | ||
| } | ||
| rearrangedExpressions | ||
| } | ||
|
|
||
| private def expressionBuilder[T <: ExpressionBuilder : ClassTag]( | ||
| name: String, | ||
| builder: T, | ||
|
|
@@ -902,7 +931,8 @@ object FunctionRegistry { | |
| val info = FunctionRegistryBase.expressionInfo[T](name, since) | ||
| val funcBuilder = (expressions: Seq[Expression]) => { | ||
| assert(expressions.forall(_.resolved), "function arguments must be resolved.") | ||
| val expr = builder.build(name, expressions) | ||
| val rearrangedExpressions = rearrangeExpressions(name, builder, expressions) | ||
| val expr = builder.build(name, rearrangedExpressions) | ||
| if (setAlias) expr.setTagValue(FUNC_ALIAS, name) | ||
| expr | ||
| } | ||
|
|
@@ -935,9 +965,22 @@ object FunctionRegistry { | |
|
|
||
| private def expressionGeneratorOuter[T <: Generator : ClassTag](name: String) | ||
| : (String, (ExpressionInfo, FunctionBuilder)) = { | ||
| val (_, (info, generatorBuilder)) = expression[T](name) | ||
| val (_, (info, builder)) = expression[T](name) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It this change necessary?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I felt it was because the name I created had the exact same name as the variable above and I didn't want to cause confusion. Also, in other example functions, builder is the variable name used for this kind of object, so I thought it be good to clear up the ambigiuity. |
||
| val outerBuilder = (args: Seq[Expression]) => { | ||
| GeneratorOuter(generatorBuilder(args).asInstanceOf[Generator]) | ||
| GeneratorOuter(builder(args).asInstanceOf[Generator]) | ||
| } | ||
| (name, (info, outerBuilder)) | ||
| } | ||
|
|
||
| private def expressionGeneratorBuilderOuter[T <: ExpressionBuilder : ClassTag] | ||
| (name: String, builder: T) : (String, (ExpressionInfo, FunctionBuilder)) = { | ||
| val info = FunctionRegistryBase.expressionInfo[T](name, since = None) | ||
| val outerBuilder = (args: Seq[Expression]) => { | ||
| val rearrangedArgs = | ||
| FunctionRegistry.rearrangeExpressions(name, builder, args) | ||
| val generator = builder.build(name, rearrangedArgs) | ||
| assert(generator.isInstanceOf[Generator]) | ||
| GeneratorOuter(generator.asInstanceOf[Generator]) | ||
| } | ||
| (name, (info, outerBuilder)) | ||
| } | ||
|
|
@@ -980,6 +1023,30 @@ object TableFunctionRegistry { | |
| (name, (info, (expressions: Seq[Expression]) => builder(expressions))) | ||
| } | ||
|
|
||
| /** | ||
| * A function used for table-valued functions to return a builder that | ||
| * when given input arguments, will return a function expression representing | ||
| * the table-valued functions. | ||
| * | ||
| * @param name Name of the function | ||
| * @param builder Object which will build the expression given input arguments | ||
| * @param since Time of implementation | ||
| * @tparam T Type of the builder | ||
| * @return A tuple of the function name, expression info, and function builder | ||
| */ | ||
| def generatorBuilder[T <: GeneratorBuilder : ClassTag]( | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. please add a comment for this method |
||
| name: String, | ||
| builder: T, | ||
| since: Option[String] = None): (String, (ExpressionInfo, TableFunctionBuilder)) = { | ||
| val info = FunctionRegistryBase.expressionInfo[T](name, since) | ||
| val funcBuilder = (expressions: Seq[Expression]) => { | ||
| assert(expressions.forall(_.resolved), "function arguments must be resolved.") | ||
| val rearrangedExpressions = FunctionRegistry.rearrangeExpressions(name, builder, expressions) | ||
| builder.build(name, rearrangedExpressions) | ||
| } | ||
| (name, (info, funcBuilder)) | ||
| } | ||
|
|
||
| def generator[T <: Generator : ClassTag](name: String, outer: Boolean = false) | ||
| : (String, (ExpressionInfo, TableFunctionBuilder)) = { | ||
| val (info, builder) = FunctionRegistryBase.build[T](name, since = None) | ||
|
|
@@ -999,8 +1066,8 @@ object TableFunctionRegistry { | |
|
|
||
| val logicalPlans: Map[String, (ExpressionInfo, TableFunctionBuilder)] = Map( | ||
| logicalPlan[Range]("range"), | ||
| generator[Explode]("explode"), | ||
| generator[Explode]("explode_outer", outer = true), | ||
| generatorBuilder("explode", ExplodeGeneratorBuilder), | ||
| generatorBuilder("explode_outer", ExplodeOuterGeneratorBuilder), | ||
| generator[Inline]("inline"), | ||
| generator[Inline]("inline_outer", outer = true), | ||
| generator[JsonTuple]("json_tuple"), | ||
|
|
@@ -1022,6 +1089,28 @@ object TableFunctionRegistry { | |
| val functionSet: Set[FunctionIdentifier] = builtin.listFunction().toSet | ||
| } | ||
|
|
||
| trait ExpressionBuilder { | ||
| def build(funcName: String, expressions: Seq[Expression]): Expression | ||
| /** | ||
| * This is a trait used for scalar valued functions that defines how their expression | ||
| * representations are constructed in [[FunctionRegistry]]. | ||
| */ | ||
| trait ExpressionBuilder extends FunctionBuilderBase[Expression] | ||
|
|
||
| /** | ||
| * This is a trait used for table valued functions that defines how their expression | ||
| * representations are constructed in [[TableFunctionRegistry]]. | ||
| */ | ||
| trait GeneratorBuilder extends FunctionBuilderBase[LogicalPlan] { | ||
| override final def build(funcName: String, expressions: Seq[Expression]) : LogicalPlan = { | ||
| Generate( | ||
| buildGenerator(funcName, expressions), | ||
| unrequiredChildIndex = Nil, | ||
| outer = isOuter, | ||
| qualifier = None, | ||
| generatorOutput = Nil, | ||
| child = OneRowRelation()) | ||
| } | ||
|
|
||
| def isOuter: Boolean | ||
|
|
||
| def buildGenerator(funcName: String, expressions: Seq[Expression]) : Generator | ||
learningchess2003 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.