Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions common/utils/src/main/resources/error/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -666,6 +666,7 @@ The following SQLSTATEs are collated from:
|4274C |42 |Syntax Error or Access Rule Violation |74C |The specified attribute was not found in the trusted context.|DB2 |N |DB2 |
|4274D |42 |Syntax Error or Access Rule Violation |74D |The specified attribute already exists in the trusted context.|DB2 |N |DB2 |
|4274E |42 |Syntax Error or Access Rule Violation |74E |The specified attribute is not supported in the trusted context.|DB2 |N |DB2 |
|4274K |42 |Syntax Error or Access Rule Violation |74K |Invalid use of a named argument when invoking a routine.|DB2 |N |DB2 |
|4274M |42 |Syntax Error or Access Rule Violation |74M |An undefined period name was detected. |DB2 |N |DB2 |
|42801 |42 |Syntax Error or Access Rule Violation |801 |Isolation level UR is invalid, because the result table is not read-only.|DB2 |N |DB2 |
|42802 |42 |Syntax Error or Access Rule Violation |802 |The number of target values is not the same as the number of source values.|DB2 |N |DB2 |
Expand Down
44 changes: 43 additions & 1 deletion common/utils/src/main/resources/error/error-classes.json
Original file line number Diff line number Diff line change
Expand Up @@ -738,6 +738,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>."
Expand Down Expand Up @@ -1956,7 +1974,13 @@
"Not allowed to implement multiple UDF interfaces, UDF class <className>."
]
},
"NAMED_ARGUMENTS_SUPPORT_DISABLED" : {
"NAMED_PARAMETERS_NOT_SUPPORTED" : {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: The object is named as object NamedArgumentsSupport. Could you unify the naming on the 'parameters' and 'arguments'?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nvm, it seems reasonable for the naming of NamedArgumentsSupport but NAMED_PARAMETERS_NOT_SUPPORTED.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Serge was actually the one who recommended the name change. It should be fine.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Right, I agree with the change in the error class to parameter, but I was thinking if similar changes should be applied on NamedArgumentsSupport, e.g. NamedParametersSupport.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Okay, that makes sense.

"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."
]
Expand Down Expand Up @@ -2295,6 +2319,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>."
Expand Down Expand Up @@ -2485,6 +2515,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>."
Expand Down Expand Up @@ -2514,6 +2550,12 @@
],
"sqlState" : "428C4"
},
"UNRECOGNIZED_PARAMETER_NAME" : {
"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>."
Expand Down
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.


34 changes: 33 additions & 1 deletion docs/sql-error-conditions.md
Original file line number Diff line number Diff line change
Expand Up @@ -456,6 +456,14 @@ Found duplicate clauses: `<clauseName>`. Please, remove one of them.

Found duplicate keys `<keyColumn>`.

### [DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT](sql-error-conditions-duplicate-routine-parameter-assignment-error-class.html)

[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>`.

For more details see [DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT](sql-error-conditions-duplicate-routine-parameter-assignment-error-class.html)

### EMPTY_JSON_FIELD_VALUE

[SQLSTATE: 42604](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
Expand Down Expand Up @@ -1210,7 +1218,13 @@ SQLSTATE: none assigned

Not allowed to implement multiple UDF interfaces, UDF class `<className>`.

### NAMED_ARGUMENTS_SUPPORT_DISABLED
### NAMED_PARAMETERS_NOT_SUPPORTED

[SQLSTATE: 4274K](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)

Named parameters are not supported for function `<functionName>`; please retry the query with positional arguments to the function call instead.

### NAMED_PARAMETER_SUPPORT_DISABLED

SQLSTATE: none assigned

Expand Down Expand Up @@ -1521,6 +1535,12 @@ Failed to rename as `<sourcePath>` was not found.

The `<clause>` clause may be used at most once per `<operation>` operation.

### REQUIRED_PARAMETER_NOT_FOUND

[SQLSTATE: 4274K](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)

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.

### REQUIRES_SINGLE_PART_NAMESPACE

[SQLSTATE: 42K05](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
Expand Down Expand Up @@ -1724,6 +1744,12 @@ Found an unclosed bracketed comment. Please, append */ at the end of the comment

Parameter `<paramIndex>` of function `<functionName>` requires the `<requiredType>` type, however `<inputSql>` has the type `<inputType>`.

### UNEXPECTED_POSITIONAL_ARGUMENT

[SQLSTATE: 4274K](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)

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.

### UNKNOWN_PROTOBUF_MESSAGE_TYPE

SQLSTATE: none assigned
Expand Down Expand Up @@ -1754,6 +1780,12 @@ Unpivot value columns must share a least common type, some types do not: [`<type

All unpivot value columns must have the same size as there are value column names (`<names>`).

### UNRECOGNIZED_PARAMETER_NAME

[SQLSTATE: 4274K](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)

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>`].

### UNRECOGNIZED_SQL_TYPE

[SQLSTATE: 42704](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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._
Expand Down Expand Up @@ -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"),
Expand Down Expand Up @@ -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),
Expand Down Expand Up @@ -823,7 +823,7 @@ object FunctionRegistry {
castAlias("string", StringType),

// mask functions
expression[Mask]("mask"),
expressionBuilder("mask", MaskExpressionBuilder),

// csv
expression[CsvToStructs]("from_csv"),
Expand Down Expand Up @@ -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,
Expand All @@ -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
}
Expand Down Expand Up @@ -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)
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))
}
Expand Down Expand Up @@ -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](
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)
Expand All @@ -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"),
Expand All @@ -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
}
Loading