You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "learningchess2003 (via GitHub)" <gi...@apache.org> on 2023/07/05 15:16:57 UTC

[GitHub] [spark] learningchess2003 opened a new pull request, #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

learningchess2003 opened a new pull request, #41864:
URL: https://github.com/apache/spark/pull/41864

   ### What changes were proposed in this pull request?
   Add analyzer support for named function arguments. 
   
   ### Why are the changes needed?
   Part of the project needed for general named function argument support.
   
   ### Does this PR introduce _any_ user-facing change?
   We added support for named arguments for the ```CountMinSketchAgg``` and ```Mask``` SQL functions.
   
   ### How was this patch tested?
   A new suite was added for this test called NamedArgumentFunctionSuite.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] anchovYu commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "anchovYu (via GitHub)" <gi...@apache.org>.
anchovYu commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1253731057


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -674,6 +674,12 @@
     ],
     "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>."
+    ],
+    "sqlState" : "4274K"

Review Comment:
   Let's clarify the sqlState with @srielau . The sqlState used in the error classes should be recorded in https://github.com/apache/spark/tree/master/common/utils/src/main/resources/error. According to https://www.ibm.com/docs/en/i/7.4?topic=codes-listing-sqlstate-values the `4274K` seems quite valid, should we document it in the doc https://github.com/apache/spark/tree/master/common/utils/src/main/resources/error?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1262224937


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -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" : {
+       "POSITIONAL_AND_NAMED_ARGUMENT_DOUBLE_REFERENCE" : {
+         "message" : [
+           "A positional argument and named argument both referred to the same parameter."
+         ]
+       },
+      "DOUBLE_NAMED_ARGUMENT_REFERENCE" : {

Review Comment:
   ```suggestion
         "MULTI_NAMED_ARGUMENT_REFERENCE" : {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] anchovYu commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "anchovYu (via GitHub)" <gi...@apache.org>.
anchovYu commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1258916558


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala:
##########
@@ -344,4 +346,13 @@ object Mask extends SupportsNamedArguments {
       strArg, upperCharArg, lowerCharArg, digitCharArg, otherCharArg))
     Seq(functionSignature)
   }
+
+  override def build(funcName: String, expressions: Seq[Expression]): Expression = {
+    if (expressions.length < 1 || expressions.length > 5) {
+      throw QueryCompilationErrors.wrongNumArgsError(
+        funcName, Seq(1, 2, 3, 4, 5), expressions.length)
+    }
+    new Mask(expressions(0), expressions(1), expressions(2), expressions(3), expressions(4))

Review Comment:
   I think we should clearly define our plan on new functions that supports named arguments - what is our behavior when users use all positional arguments? New functions may only have one constructor without overloarding but with well defined function signatures, e.g. Seq(`a` required, `b` optional). If users only pass func(1), do we throw exception that it does not match the argument count, or do our framework just consider this 1 as `a`, and fill the `b` with default value?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1261774415


##########
sql/core/src/test/resources/sql-tests/results/named-function-arguments.sql.out:
##########
@@ -2,121 +2,258 @@
 -- !query
 SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd')
 -- !query schema
+struct<mask(AbCD123-@$#, Q, q, d, o):string>
+-- !query output
+QqQQdddoooo
+
+
+-- !query
+SELECT mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbCD123-@$#')
+-- !query schema
+struct<mask(AbCD123-@$#, Q, q, d, o):string>
+-- !query output
+QqQQdddoooo
+
+
+-- !query
+SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', digitChar => 'd')
+-- !query schema
+struct<mask(AbCD123-@$#, Q, q, d, NULL):string>
+-- !query output
+QqQQddd-@$#
+
+
+-- !query
+SELECT mask(lowerChar => 'q', upperChar => 'Q', digitChar => 'd', str => 'AbCD123-@$#')
+-- !query schema
+struct<mask(AbCD123-@$#, Q, q, d, NULL):string>
+-- !query output
+QqQQddd-@$#
+
+
+-- !query
+create temporary view t2 as select * from values
+  ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'),
+  ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'),
+  ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null),
+  ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'),
+  ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'),
+  ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'),
+  ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'),
+  ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null)
+  as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT hex(count_min_sketch(t2d, seed => 1, epsilon => 0.5d, confidence => 0.5d)) FROM t2
+-- !query schema
+struct<hex(count_min_sketch(t2d, 0.5, 0.5, 1)):string>
+-- !query output
+00000001000000000000000D0000000100000004000000005D8D6AB90000000000000002000000000000000700000000000000010000000000000003
+
+
+-- !query
+SELECT hex(count_min_sketch(seed => 1, epsilon => 0.5d, confidence => 0.5d, column => t2d)) FROM t2
+-- !query schema
+struct<hex(count_min_sketch(t2d, 0.5, 0.5, 1)):string>
+-- !query output
+00000001000000000000000D0000000100000004000000005D8D6AB90000000000000002000000000000000700000000000000010000000000000003
+
+
+-- !query
+SELECT hex(count_min_sketch(t2d, 0.5d, seed => 1, confidence => 0.5d)) FROM t2
+-- !query schema
+struct<hex(count_min_sketch(t2d, 0.5, 0.5, 1)):string>
+-- !query output
+00000001000000000000000D0000000100000004000000005D8D6AB90000000000000002000000000000000700000000000000010000000000000003
+
+
+-- !query
+SELECT * FROM explode(collection => array(1, 2))
+-- !query schema
+struct<col:int>
+-- !query output
+1
+2
+
+
+-- !query
+SELECT * FROM explode_outer(collection => map('a', 1, 'b', 2))
+-- !query schema
+struct<key:string,value:int>
+-- !query output
+a	1
+b	2
+
+
+-- !query
+CREATE OR REPLACE TEMPORARY VIEW v AS SELECT id FROM range(0, 8)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * FROM explode(collection => TABLE v)
+-- !query schema
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT",
+  "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
   "sqlState" : "42K09",
   "messageParameters" : {
-    "inputExpr" : "\"namedargumentexpression(q)\"",
-    "inputName" : "upperChar",
-    "inputType" : "\"STRING\"",
-    "sqlExpr" : "\"mask(AbCD123-@$#, namedargumentexpression(q), namedargumentexpression(Q), namedargumentexpression(o), namedargumentexpression(d))\""
+    "inputSql" : "\"outer(__auto_generated_subquery_name_0.c)\"",
+    "inputType" : "\"STRUCT<id: BIGINT>\"",
+    "paramIndex" : "1",
+    "requiredType" : "(\"ARRAY\" or \"MAP\")",
+    "sqlExpr" : "\"explode(outer(__auto_generated_subquery_name_0.c))\""
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 15,
+    "stopIndex" : 44,
+    "fragment" : "explode(collection => TABLE v)"
+  } ]
+}
+
+
+-- !query
+SELECT mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "UNEXPECTED_POSITIONAL_ARGUMENT",
+  "sqlState" : "4274K",
+  "messageParameters" : {
+    "functionName" : "`mask`"
   },
   "queryContext" : [ {
     "objectType" : "",
     "objectName" : "",
     "startIndex" : 8,
     "stopIndex" : 98,
-    "fragment" : "mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd')"
+    "fragment" : "mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd')"
   } ]
 }
 
 
 -- !query
-SELECT mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbCD123-@$#')
+SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', digitChar => 'e')
 -- !query schema
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT",
-  "sqlState" : "42K09",
+  "errorClass" : "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.DOUBLE_NAMED_ARGUMENT_REFERENCE",
+  "sqlState" : "4274K",
   "messageParameters" : {
-    "inputExpr" : "\"namedargumentexpression(Q)\"",
-    "inputName" : "upperChar",
-    "inputType" : "\"STRING\"",
-    "sqlExpr" : "\"mask(namedargumentexpression(q), namedargumentexpression(Q), namedargumentexpression(o), namedargumentexpression(d), namedargumentexpression(AbCD123-@$#))\""
+    "functionName" : "`mask`",
+    "parameterName" : "`digitChar`"
   },
   "queryContext" : [ {
     "objectType" : "",
     "objectName" : "",
     "startIndex" : 8,
-    "stopIndex" : 105,
-    "fragment" : "mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbCD123-@$#')"
+    "stopIndex" : 116,
+    "fragment" : "mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', digitChar => 'e')"
   } ]
 }
 
 
 -- !query
-SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', digitChar => 'd')
+SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbC')
 -- !query schema
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT",
-  "sqlState" : "42K09",
+  "errorClass" : "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.POSITIONAL_AND_NAMED_ARGUMENT_DOUBLE_REFERENCE",
+  "sqlState" : "4274K",
   "messageParameters" : {
-    "inputExpr" : "\"namedargumentexpression(q)\"",
-    "inputName" : "upperChar",
-    "inputType" : "\"STRING\"",
-    "sqlExpr" : "\"mask(AbCD123-@$#, namedargumentexpression(q), namedargumentexpression(Q), namedargumentexpression(d), NULL)\""
+    "functionName" : "`mask`",
+    "parameterName" : "`str`"
   },
   "queryContext" : [ {
     "objectType" : "",
     "objectName" : "",
     "startIndex" : 8,
-    "stopIndex" : 80,
-    "fragment" : "mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', digitChar => 'd')"
+    "stopIndex" : 112,
+    "fragment" : "mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbC')"
   } ]
 }
 
 
 -- !query
-SELECT mask(lowerChar => 'q', upperChar => 'Q', digitChar => 'd', str => 'AbCD123-@$#')
+SELECT mask(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",
+  "errorClass" : "REQUIRED_PARAMETER_NOT_FOUND",
+  "sqlState" : "4274K",
   "messageParameters" : {
-    "inputExpr" : "\"namedargumentexpression(Q)\"",
-    "inputName" : "upperChar",
-    "inputType" : "\"STRING\"",
-    "sqlExpr" : "\"mask(namedargumentexpression(q), namedargumentexpression(Q), namedargumentexpression(d), namedargumentexpression(AbCD123-@$#), NULL)\""
+    "functionName" : "`mask`",
+    "parameterName" : "`str`"
   },
   "queryContext" : [ {
     "objectType" : "",
     "objectName" : "",
     "startIndex" : 8,
-    "stopIndex" : 87,
-    "fragment" : "mask(lowerChar => 'q', upperChar => 'Q', digitChar => 'd', str => 'AbCD123-@$#')"
+    "stopIndex" : 83,
+    "fragment" : "mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd')"
   } ]
 }
 
 
 -- !query
-SELECT mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd')
+SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', cellular => 'automata')
 -- !query schema
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE",
-  "sqlState" : "42K09",
+  "errorClass" : "UNRECOGNIZED_PARAMETER_NAME",
+  "sqlState" : "4274K",
   "messageParameters" : {
-    "exprName" : "upperChar",
-    "sqlExpr" : "\"mask(namedargumentexpression(q), AbCD123-@$#, namedargumentexpression(Q), namedargumentexpression(o), namedargumentexpression(d))\""
+    "argumentName" : "`cellular`",
+    "functionName" : "`mask`",
+    "proposal" : "`str` `upperChar` `otherChar` "
   },
   "queryContext" : [ {
     "objectType" : "",
     "objectName" : "",
     "startIndex" : 8,
-    "stopIndex" : 98,
-    "fragment" : "mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd')"
+    "stopIndex" : 122,
+    "fragment" : "mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', cellular => 'automata')"
   } ]
 }
+
+
+-- !query
+SELECT encode(str => 'a', charset => 'utf-8')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkException
+{
+  "errorClass" : "INTERNAL_ERROR",
+  "sqlState" : "XX000",
+  "messageParameters" : {
+    "message" : "Cannot generate code for expression: str => a"

Review Comment:
   Oh, this needs to be fixed. This should definitely be NAMED_ARGUMENTS_NOT_SUPPORTED. I don't think I've done that yet with this framework. Will need to do that.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1254942120


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -894,8 +896,10 @@ object FunctionRegistry {
       since: Option[String] = None): (String, (ExpressionInfo, FunctionBuilder)) = {
     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 =
+        SupportsNamedArguments.getRearrangedExpressions[T](expressions, name)

Review Comment:
   I see. In that case, I can delete it. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1253625316


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -674,6 +674,12 @@
     ],
     "sqlState" : "23505"
   },
+  "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT" : {
+    "message" : [
+      "Call to function <functionName> is invalid because it includes multiple argument assignments to the same name <parameterName>."

Review Comment:
   ```suggestion
         "Call to function <functionName> is invalid because it includes multiple argument assignments to the same parameter name <parameterName>."
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala:
##########
@@ -320,4 +321,18 @@ object Mask {
       case _ => maskedChar(c, maskOther)
     }
   }
+  override def functionSignatures: Seq[FunctionSignature] = {

Review Comment:
   ```suggestion
   
     override def functionSignatures: Seq[FunctionSignature] = {
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+    dataType: NamedArgumentType,
+    default: Option[Expression] = None)
+
+/**
+ * Represents a method signature and the list of arguments it receives as input.
+ * Currently, overloads are not supported and only one FunctionSignature is allowed
+ * per function expression.
+ *
+ * @param parameters The list of arguments which the function takes
+ */
+case class FunctionSignature(parameters: Seq[NamedArgument])
+
+/**
+ * The class which companion objects of function expression implement to
+ * support named arguments for that function expression.

Review Comment:
   Can you add a short example here as well?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+    dataType: NamedArgumentType,
+    default: Option[Expression] = None)
+
+/**
+ * Represents a method signature and the list of arguments it receives as input.
+ * Currently, overloads are not supported and only one FunctionSignature is allowed
+ * per function expression.
+ *
+ * @param parameters The list of arguments which the function takes
+ */
+case class FunctionSignature(parameters: Seq[NamedArgument])
+
+/**
+ * The class which companion objects of function expression implement to
+ * support named arguments for that function expression.
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use the provided version rearrange as it is consistent with
+   * the SQL standard. If absolutely necessary the developer can choose to override the default
+   * behavior for additional flexibility.
+   *
+   * @param functionSignature Function signature that denotes positional order of arguments
+   * @param args The sequence of expressions from function invocation
+   * @param functionName The name of the function invoked for debugging purposes
+   * @return positional order of arguments according to FunctionSignature
+   */
+  protected def rearrange(functionSignature: FunctionSignature,

Review Comment:
   please start this on the next line with an indent of +4 spaces per the style guide



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+    dataType: NamedArgumentType,
+    default: Option[Expression] = None)
+
+/**
+ * Represents a method signature and the list of arguments it receives as input.
+ * Currently, overloads are not supported and only one FunctionSignature is allowed
+ * per function expression.
+ *
+ * @param parameters The list of arguments which the function takes
+ */
+case class FunctionSignature(parameters: Seq[NamedArgument])
+
+/**
+ * The class which companion objects of function expression implement to
+ * support named arguments for that function expression.
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use the provided version rearrange as it is consistent with
+   * the SQL standard. If absolutely necessary the developer can choose to override the default
+   * behavior for additional flexibility.
+   *
+   * @param functionSignature Function signature that denotes positional order of arguments
+   * @param args The sequence of expressions from function invocation
+   * @param functionName The name of the function invoked for debugging purposes
+   * @return positional order of arguments according to FunctionSignature
+   */
+  protected def rearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(functionSignature, args, functionName)
+  }
+}
+
+object SupportsNamedArguments {
+
+  /**
+   * Given a generic type, we check if the companion object of said type exists.
+   * If that object extends the trait [[SupportsNamedArguments]], then we rearrange
+   * the expressions in the order specified by the object.
+   *
+   * It is here we resubstitute [[Unevaluable]] [[NamedArgumentExpression]]s with
+   * normal expressions. This method will produce an positional argument list which
+   * is equivalent to the original argumnet list, except the expressions are now
+   * fit for consumption by [[ResolveFunctions]]
+   *
+   * @param expressions The list of positional and named argument expressions
+   * @tparam T The actual expression class.
+   * @return positional argument list
+   */
+  final def getRearrangedExpressions[T <: Expression : ClassTag](
+      expressions: Seq[Expression], functionName: String): Seq[Expression] = {
+
+    if (!expressions.exists(_.isInstanceOf[NamedArgumentExpression])) {
+      return expressions
+    }
+
+    import scala.reflect.runtime.currentMirror
+
+    // This code heavily utilizes Scala reflection which is unfamiliar to most developers.
+    // Here are the steps of this function:
+    // 1. Obtain the module symbol for the companion object of the function expression.
+    // 2. Obtain the module class symbol that represents the companion object.
+    // 3. Check if the base classes of the module class symbol contains SupportsNamedArguments.
+    //    This checks if the companion object is an implementor of SupportsNamedArguments.
+    // 4. Check if the module class symbol is a top level object. Reflection is unable to
+    //    obtain a companion object instance if it is member of some enclosing class unless
+    //    instance of said enclosing class is provided which we do not have.
+    // 5. Use reflection to obtain instance of companion object and perform immediate cast to
+    //    SupportsNamedArguments as it is already verified the cast is safe.
+    // 6. Obtain function signature and rearrange expression according to the given signature.
+    val runtimeClass = scala.reflect.classTag[T].runtimeClass
+    val targetModuleSymbol = currentMirror.classSymbol(runtimeClass).companion
+    val parentClass = scala.reflect.classTag[SupportsNamedArguments].runtimeClass
+    val parentSymbol = currentMirror.classSymbol(parentClass)
+
+    targetModuleSymbol match {
+      case scala.reflect.runtime.universe.NoSymbol =>

Review Comment:
   you can just check `if (targetModuleSymbol.isInstanceOf[scala.reflect.runtime.universe.NoSymbol])` and throw the exception in that case, removing the `case _` and de-denting the rest of the block.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+    dataType: NamedArgumentType,
+    default: Option[Expression] = None)
+
+/**
+ * Represents a method signature and the list of arguments it receives as input.
+ * Currently, overloads are not supported and only one FunctionSignature is allowed
+ * per function expression.
+ *
+ * @param parameters The list of arguments which the function takes
+ */
+case class FunctionSignature(parameters: Seq[NamedArgument])
+
+/**
+ * The class which companion objects of function expression implement to
+ * support named arguments for that function expression.
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use the provided version rearrange as it is consistent with
+   * the SQL standard. If absolutely necessary the developer can choose to override the default
+   * behavior for additional flexibility.
+   *
+   * @param functionSignature Function signature that denotes positional order of arguments
+   * @param args The sequence of expressions from function invocation
+   * @param functionName The name of the function invoked for debugging purposes
+   * @return positional order of arguments according to FunctionSignature
+   */
+  protected def rearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(functionSignature, args, functionName)
+  }
+}
+
+object SupportsNamedArguments {
+
+  /**
+   * Given a generic type, we check if the companion object of said type exists.
+   * If that object extends the trait [[SupportsNamedArguments]], then we rearrange
+   * the expressions in the order specified by the object.
+   *
+   * It is here we resubstitute [[Unevaluable]] [[NamedArgumentExpression]]s with
+   * normal expressions. This method will produce an positional argument list which
+   * is equivalent to the original argumnet list, except the expressions are now
+   * fit for consumption by [[ResolveFunctions]]
+   *
+   * @param expressions The list of positional and named argument expressions
+   * @tparam T The actual expression class.
+   * @return positional argument list
+   */
+  final def getRearrangedExpressions[T <: Expression : ClassTag](
+      expressions: Seq[Expression], functionName: String): Seq[Expression] = {
+
+    if (!expressions.exists(_.isInstanceOf[NamedArgumentExpression])) {
+      return expressions
+    }
+
+    import scala.reflect.runtime.currentMirror
+
+    // This code heavily utilizes Scala reflection which is unfamiliar to most developers.
+    // Here are the steps of this function:
+    // 1. Obtain the module symbol for the companion object of the function expression.
+    // 2. Obtain the module class symbol that represents the companion object.
+    // 3. Check if the base classes of the module class symbol contains SupportsNamedArguments.
+    //    This checks if the companion object is an implementor of SupportsNamedArguments.
+    // 4. Check if the module class symbol is a top level object. Reflection is unable to
+    //    obtain a companion object instance if it is member of some enclosing class unless
+    //    instance of said enclosing class is provided which we do not have.
+    // 5. Use reflection to obtain instance of companion object and perform immediate cast to
+    //    SupportsNamedArguments as it is already verified the cast is safe.
+    // 6. Obtain function signature and rearrange expression according to the given signature.
+    val runtimeClass = scala.reflect.classTag[T].runtimeClass
+    val targetModuleSymbol = currentMirror.classSymbol(runtimeClass).companion
+    val parentClass = scala.reflect.classTag[SupportsNamedArguments].runtimeClass
+    val parentSymbol = currentMirror.classSymbol(parentClass)
+
+    targetModuleSymbol match {
+      case scala.reflect.runtime.universe.NoSymbol =>
+        throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
+      case _ =>
+        val moduleClassSymbol = targetModuleSymbol.asModule.moduleClass.asClass
+        if (moduleClassSymbol.baseClasses.contains(parentSymbol)) {
+          if (currentMirror.runtimeClass(moduleClassSymbol).getEnclosingClass != null) {
+            throw QueryCompilationErrors.cannotObtainCompanionObjectInstance(functionName)
+          }
+          val instance = currentMirror.reflectModule(targetModuleSymbol.asModule)
+            .instance.asInstanceOf[SupportsNamedArguments]
+          if (instance.functionSignatures.size != 1) {
+            throw QueryCompilationErrors.multipleFunctionSignatures(
+              functionName, instance.functionSignatures)
+          }
+          instance.rearrange(instance.functionSignatures.head, expressions, functionName)
+        } else {
+          throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
+        }
+    }
+  }
+
+  // Exposed for testing
+  final def defaultRearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters = functionSignature.parameters

Review Comment:
   please add explicit types for these `val`s for better readability?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -50,6 +50,65 @@ import org.apache.spark.sql.types._
  */
 private[sql] object QueryCompilationErrors extends QueryErrorsBase {
 
+  def cannotObtainCompanionObjectInstance(functionName: String): Throwable = {
+    SparkException.internalError(s"Cannot obtain companion object for " +
+      s"function expression: $functionName. Companion must be top-level object.")

Review Comment:
   ```suggestion
         s"function expression: $functionName. Please note that this companion must be a top-level object.")
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+    dataType: NamedArgumentType,
+    default: Option[Expression] = None)
+
+/**
+ * Represents a method signature and the list of arguments it receives as input.
+ * Currently, overloads are not supported and only one FunctionSignature is allowed
+ * per function expression.
+ *
+ * @param parameters The list of arguments which the function takes
+ */
+case class FunctionSignature(parameters: Seq[NamedArgument])
+
+/**
+ * The class which companion objects of function expression implement to
+ * support named arguments for that function expression.
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use the provided version rearrange as it is consistent with
+   * the SQL standard. If absolutely necessary the developer can choose to override the default
+   * behavior for additional flexibility.
+   *
+   * @param functionSignature Function signature that denotes positional order of arguments
+   * @param args The sequence of expressions from function invocation
+   * @param functionName The name of the function invoked for debugging purposes
+   * @return positional order of arguments according to FunctionSignature
+   */
+  protected def rearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(functionSignature, args, functionName)
+  }
+}
+
+object SupportsNamedArguments {
+
+  /**
+   * Given a generic type, we check if the companion object of said type exists.
+   * If that object extends the trait [[SupportsNamedArguments]], then we rearrange
+   * the expressions in the order specified by the object.
+   *
+   * It is here we resubstitute [[Unevaluable]] [[NamedArgumentExpression]]s with
+   * normal expressions. This method will produce an positional argument list which
+   * is equivalent to the original argumnet list, except the expressions are now
+   * fit for consumption by [[ResolveFunctions]]
+   *
+   * @param expressions The list of positional and named argument expressions
+   * @tparam T The actual expression class.
+   * @return positional argument list
+   */
+  final def getRearrangedExpressions[T <: Expression : ClassTag](
+      expressions: Seq[Expression], functionName: String): Seq[Expression] = {
+
+    if (!expressions.exists(_.isInstanceOf[NamedArgumentExpression])) {
+      return expressions
+    }
+
+    import scala.reflect.runtime.currentMirror
+
+    // This code heavily utilizes Scala reflection which is unfamiliar to most developers.
+    // Here are the steps of this function:
+    // 1. Obtain the module symbol for the companion object of the function expression.
+    // 2. Obtain the module class symbol that represents the companion object.
+    // 3. Check if the base classes of the module class symbol contains SupportsNamedArguments.
+    //    This checks if the companion object is an implementor of SupportsNamedArguments.
+    // 4. Check if the module class symbol is a top level object. Reflection is unable to
+    //    obtain a companion object instance if it is member of some enclosing class unless
+    //    instance of said enclosing class is provided which we do not have.
+    // 5. Use reflection to obtain instance of companion object and perform immediate cast to
+    //    SupportsNamedArguments as it is already verified the cast is safe.
+    // 6. Obtain function signature and rearrange expression according to the given signature.
+    val runtimeClass = scala.reflect.classTag[T].runtimeClass
+    val targetModuleSymbol = currentMirror.classSymbol(runtimeClass).companion
+    val parentClass = scala.reflect.classTag[SupportsNamedArguments].runtimeClass
+    val parentSymbol = currentMirror.classSymbol(parentClass)
+
+    targetModuleSymbol match {
+      case scala.reflect.runtime.universe.NoSymbol =>
+        throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
+      case _ =>
+        val moduleClassSymbol = targetModuleSymbol.asModule.moduleClass.asClass
+        if (moduleClassSymbol.baseClasses.contains(parentSymbol)) {
+          if (currentMirror.runtimeClass(moduleClassSymbol).getEnclosingClass != null) {
+            throw QueryCompilationErrors.cannotObtainCompanionObjectInstance(functionName)
+          }
+          val instance = currentMirror.reflectModule(targetModuleSymbol.asModule)
+            .instance.asInstanceOf[SupportsNamedArguments]
+          if (instance.functionSignatures.size != 1) {
+            throw QueryCompilationErrors.multipleFunctionSignatures(
+              functionName, instance.functionSignatures)
+          }
+          instance.rearrange(instance.functionSignatures.head, expressions, functionName)
+        } else {
+          throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
+        }
+    }
+  }
+
+  // Exposed for testing
+  final def defaultRearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters = functionSignature.parameters
+    val firstNamedArgIdx = args.indexWhere(_.isInstanceOf[NamedArgumentExpression])
+    val (positionalArgs, namedArgs) = args.splitAt(firstNamedArgIdx)
+    val namedParameters = parameters.drop(positionalArgs.size)
+
+    // Performing some checking to ensure valid argument list
+    val allParameterNames: Seq[String] = parameters.map(_.name)
+    val parameterNamesSet: Set[String] = allParameterNames.toSet
+    val assignedParameterSet = collection.mutable.Set[String](
+      allParameterNames.take(positionalArgs.size): _*)
+    for (arg <- namedArgs) {
+      arg match {
+        case namedArg: NamedArgumentExpression =>
+          if (assignedParameterSet.contains(namedArg.key)) {
+            throw QueryCompilationErrors.duplicateRoutineParameterAssignment(
+              functionName, namedArg.key)
+          }
+          if (!parameterNamesSet.contains(namedArg.key)) {
+            throw QueryCompilationErrors.unrecognizedParameterName(functionName, namedArg.key)
+          }
+          assignedParameterSet.add(namedArg.key)
+        case _ =>
+          throw QueryCompilationErrors.unexpectedPositionalArgument(functionName)
+      }
+    }
+
+    // Construct a map from argument name to value for argument rearrangement
+    val namedArgMap = namedArgs.map { arg =>
+      val namedArg = arg.asInstanceOf[NamedArgumentExpression]
+      namedArg.key -> namedArg.value
+    }.toMap
+
+    // Rearrange named arguments to match their positional order
+    val rearrangedNamedArgs = namedParameters.map { param =>
+      namedArgMap.getOrElse(
+        param.name,
+        if (param.default.isEmpty) {
+          throw QueryCompilationErrors.requiredParameterNotFound(functionName, param.name)
+        } else {

Review Comment:
   you can drop the `else` here since you throw an exception above.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(

Review Comment:
   Let's put this class definition first in the file, with the `NamedArgumentType` and `FixedArgumentType` following, so the reader sees the former first (object definitions in Scala may appear in any order).
   
   Also, can you please update `A named parameter` in the class comment to something more descriptive, indicating how expression classes that implement SQL functions are supposed to use this?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala:
##########
@@ -208,3 +209,13 @@ case class CountMinSketchAgg(
       confidenceExpression = third,
       seedExpression = fourth)
 }
+object CountMinSketchAgg extends SupportsNamedArguments {

Review Comment:
   ```suggestion
   
   object CountMinSketchAgg extends SupportsNamedArguments {
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+    dataType: NamedArgumentType,
+    default: Option[Expression] = None)
+
+/**
+ * Represents a method signature and the list of arguments it receives as input.
+ * Currently, overloads are not supported and only one FunctionSignature is allowed
+ * per function expression.
+ *
+ * @param parameters The list of arguments which the function takes
+ */
+case class FunctionSignature(parameters: Seq[NamedArgument])
+
+/**
+ * The class which companion objects of function expression implement to

Review Comment:
   ```suggestion
    * The class which companion objects of function expression may implement to
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+    dataType: NamedArgumentType,
+    default: Option[Expression] = None)
+
+/**
+ * Represents a method signature and the list of arguments it receives as input.
+ * Currently, overloads are not supported and only one FunctionSignature is allowed
+ * per function expression.
+ *
+ * @param parameters The list of arguments which the function takes
+ */
+case class FunctionSignature(parameters: Seq[NamedArgument])
+
+/**
+ * The class which companion objects of function expression implement to
+ * support named arguments for that function expression.
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use the provided version rearrange as it is consistent with

Review Comment:
   ```suggestion
      * It is recommended to use this provided implementation as it is consistent with
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type

Review Comment:
   ```suggestion
    * Represents a named argument that expects a scalar value of one specific DataType.
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument

Review Comment:
   ```suggestion
    * Identifies which forms of provided argument values are expected for each
    * call to the associated SQL function.
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+    dataType: NamedArgumentType,
+    default: Option[Expression] = None)
+
+/**
+ * Represents a method signature and the list of arguments it receives as input.
+ * Currently, overloads are not supported and only one FunctionSignature is allowed
+ * per function expression.
+ *
+ * @param parameters The list of arguments which the function takes
+ */
+case class FunctionSignature(parameters: Seq[NamedArgument])
+
+/**
+ * The class which companion objects of function expression implement to
+ * support named arguments for that function expression.
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use the provided version rearrange as it is consistent with
+   * the SQL standard. If absolutely necessary the developer can choose to override the default
+   * behavior for additional flexibility.
+   *
+   * @param functionSignature Function signature that denotes positional order of arguments
+   * @param args The sequence of expressions from function invocation
+   * @param functionName The name of the function invoked for debugging purposes
+   * @return positional order of arguments according to FunctionSignature
+   */
+  protected def rearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(functionSignature, args, functionName)
+  }
+}
+
+object SupportsNamedArguments {
+
+  /**
+   * Given a generic type, we check if the companion object of said type exists.
+   * If that object extends the trait [[SupportsNamedArguments]], then we rearrange
+   * the expressions in the order specified by the object.
+   *
+   * It is here we resubstitute [[Unevaluable]] [[NamedArgumentExpression]]s with
+   * normal expressions. This method will produce an positional argument list which
+   * is equivalent to the original argumnet list, except the expressions are now
+   * fit for consumption by [[ResolveFunctions]]
+   *
+   * @param expressions The list of positional and named argument expressions
+   * @tparam T The actual expression class.
+   * @return positional argument list
+   */
+  final def getRearrangedExpressions[T <: Expression : ClassTag](
+      expressions: Seq[Expression], functionName: String): Seq[Expression] = {
+
+    if (!expressions.exists(_.isInstanceOf[NamedArgumentExpression])) {
+      return expressions
+    }
+
+    import scala.reflect.runtime.currentMirror
+
+    // This code heavily utilizes Scala reflection which is unfamiliar to most developers.
+    // Here are the steps of this function:
+    // 1. Obtain the module symbol for the companion object of the function expression.
+    // 2. Obtain the module class symbol that represents the companion object.
+    // 3. Check if the base classes of the module class symbol contains SupportsNamedArguments.
+    //    This checks if the companion object is an implementor of SupportsNamedArguments.
+    // 4. Check if the module class symbol is a top level object. Reflection is unable to
+    //    obtain a companion object instance if it is member of some enclosing class unless
+    //    instance of said enclosing class is provided which we do not have.
+    // 5. Use reflection to obtain instance of companion object and perform immediate cast to
+    //    SupportsNamedArguments as it is already verified the cast is safe.
+    // 6. Obtain function signature and rearrange expression according to the given signature.
+    val runtimeClass = scala.reflect.classTag[T].runtimeClass
+    val targetModuleSymbol = currentMirror.classSymbol(runtimeClass).companion
+    val parentClass = scala.reflect.classTag[SupportsNamedArguments].runtimeClass
+    val parentSymbol = currentMirror.classSymbol(parentClass)
+
+    targetModuleSymbol match {
+      case scala.reflect.runtime.universe.NoSymbol =>
+        throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
+      case _ =>
+        val moduleClassSymbol = targetModuleSymbol.asModule.moduleClass.asClass
+        if (moduleClassSymbol.baseClasses.contains(parentSymbol)) {

Review Comment:
   same, you can reverse the logic and throw the exception, removing the `else` clause and de-denting the rest of the block.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+    dataType: NamedArgumentType,
+    default: Option[Expression] = None)
+
+/**
+ * Represents a method signature and the list of arguments it receives as input.
+ * Currently, overloads are not supported and only one FunctionSignature is allowed
+ * per function expression.
+ *
+ * @param parameters The list of arguments which the function takes
+ */
+case class FunctionSignature(parameters: Seq[NamedArgument])
+
+/**
+ * The class which companion objects of function expression implement to
+ * support named arguments for that function expression.
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use the provided version rearrange as it is consistent with
+   * the SQL standard. If absolutely necessary the developer can choose to override the default
+   * behavior for additional flexibility.
+   *
+   * @param functionSignature Function signature that denotes positional order of arguments
+   * @param args The sequence of expressions from function invocation
+   * @param functionName The name of the function invoked for debugging purposes
+   * @return positional order of arguments according to FunctionSignature

Review Comment:
   ```suggestion
      * @return positional order of arguments according to FunctionSignature obtained
      *         by changing the order of the above provided arguments
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+    dataType: NamedArgumentType,
+    default: Option[Expression] = None)
+
+/**
+ * Represents a method signature and the list of arguments it receives as input.
+ * Currently, overloads are not supported and only one FunctionSignature is allowed
+ * per function expression.
+ *
+ * @param parameters The list of arguments which the function takes
+ */
+case class FunctionSignature(parameters: Seq[NamedArgument])
+
+/**
+ * The class which companion objects of function expression implement to
+ * support named arguments for that function expression.
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use the provided version rearrange as it is consistent with
+   * the SQL standard. If absolutely necessary the developer can choose to override the default
+   * behavior for additional flexibility.
+   *
+   * @param functionSignature Function signature that denotes positional order of arguments

Review Comment:
   Since it can be confusing after a while to read code in this area that deals with a lot of expected function signatures and provided arguments, I suggest to rename the method arguments to clarify, e.g. expectedSignature, providedArguments, functionName.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+    dataType: NamedArgumentType,
+    default: Option[Expression] = None)
+
+/**
+ * Represents a method signature and the list of arguments it receives as input.
+ * Currently, overloads are not supported and only one FunctionSignature is allowed
+ * per function expression.
+ *
+ * @param parameters The list of arguments which the function takes
+ */
+case class FunctionSignature(parameters: Seq[NamedArgument])
+
+/**
+ * The class which companion objects of function expression implement to
+ * support named arguments for that function expression.
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use the provided version rearrange as it is consistent with
+   * the SQL standard. If absolutely necessary the developer can choose to override the default
+   * behavior for additional flexibility.
+   *
+   * @param functionSignature Function signature that denotes positional order of arguments
+   * @param args The sequence of expressions from function invocation
+   * @param functionName The name of the function invoked for debugging purposes
+   * @return positional order of arguments according to FunctionSignature
+   */
+  protected def rearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(functionSignature, args, functionName)
+  }
+}
+
+object SupportsNamedArguments {
+
+  /**
+   * Given a generic type, we check if the companion object of said type exists.
+   * If that object extends the trait [[SupportsNamedArguments]], then we rearrange
+   * the expressions in the order specified by the object.
+   *
+   * It is here we resubstitute [[Unevaluable]] [[NamedArgumentExpression]]s with
+   * normal expressions. This method will produce an positional argument list which
+   * is equivalent to the original argumnet list, except the expressions are now
+   * fit for consumption by [[ResolveFunctions]]
+   *
+   * @param expressions The list of positional and named argument expressions
+   * @tparam T The actual expression class.
+   * @return positional argument list
+   */
+  final def getRearrangedExpressions[T <: Expression : ClassTag](
+      expressions: Seq[Expression], functionName: String): Seq[Expression] = {
+
+    if (!expressions.exists(_.isInstanceOf[NamedArgumentExpression])) {
+      return expressions
+    }
+
+    import scala.reflect.runtime.currentMirror
+
+    // This code heavily utilizes Scala reflection which is unfamiliar to most developers.
+    // Here are the steps of this function:
+    // 1. Obtain the module symbol for the companion object of the function expression.
+    // 2. Obtain the module class symbol that represents the companion object.
+    // 3. Check if the base classes of the module class symbol contains SupportsNamedArguments.
+    //    This checks if the companion object is an implementor of SupportsNamedArguments.
+    // 4. Check if the module class symbol is a top level object. Reflection is unable to
+    //    obtain a companion object instance if it is member of some enclosing class unless
+    //    instance of said enclosing class is provided which we do not have.
+    // 5. Use reflection to obtain instance of companion object and perform immediate cast to
+    //    SupportsNamedArguments as it is already verified the cast is safe.
+    // 6. Obtain function signature and rearrange expression according to the given signature.
+    val runtimeClass = scala.reflect.classTag[T].runtimeClass
+    val targetModuleSymbol = currentMirror.classSymbol(runtimeClass).companion
+    val parentClass = scala.reflect.classTag[SupportsNamedArguments].runtimeClass
+    val parentSymbol = currentMirror.classSymbol(parentClass)
+
+    targetModuleSymbol match {
+      case scala.reflect.runtime.universe.NoSymbol =>
+        throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
+      case _ =>
+        val moduleClassSymbol = targetModuleSymbol.asModule.moduleClass.asClass
+        if (moduleClassSymbol.baseClasses.contains(parentSymbol)) {
+          if (currentMirror.runtimeClass(moduleClassSymbol).getEnclosingClass != null) {
+            throw QueryCompilationErrors.cannotObtainCompanionObjectInstance(functionName)
+          }
+          val instance = currentMirror.reflectModule(targetModuleSymbol.asModule)
+            .instance.asInstanceOf[SupportsNamedArguments]
+          if (instance.functionSignatures.size != 1) {
+            throw QueryCompilationErrors.multipleFunctionSignatures(
+              functionName, instance.functionSignatures)
+          }
+          instance.rearrange(instance.functionSignatures.head, expressions, functionName)
+        } else {
+          throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
+        }
+    }
+  }
+
+  // Exposed for testing

Review Comment:
   you can just delete this line, it is OK for this method to be public in case others want to use it later



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+    dataType: NamedArgumentType,
+    default: Option[Expression] = None)
+
+/**
+ * Represents a method signature and the list of arguments it receives as input.
+ * Currently, overloads are not supported and only one FunctionSignature is allowed
+ * per function expression.
+ *
+ * @param parameters The list of arguments which the function takes
+ */
+case class FunctionSignature(parameters: Seq[NamedArgument])
+
+/**
+ * The class which companion objects of function expression implement to
+ * support named arguments for that function expression.
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use the provided version rearrange as it is consistent with
+   * the SQL standard. If absolutely necessary the developer can choose to override the default
+   * behavior for additional flexibility.
+   *
+   * @param functionSignature Function signature that denotes positional order of arguments
+   * @param args The sequence of expressions from function invocation
+   * @param functionName The name of the function invoked for debugging purposes
+   * @return positional order of arguments according to FunctionSignature
+   */
+  protected def rearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(functionSignature, args, functionName)
+  }
+}
+
+object SupportsNamedArguments {
+
+  /**
+   * Given a generic type, we check if the companion object of said type exists.
+   * If that object extends the trait [[SupportsNamedArguments]], then we rearrange
+   * the expressions in the order specified by the object.
+   *
+   * It is here we resubstitute [[Unevaluable]] [[NamedArgumentExpression]]s with
+   * normal expressions. This method will produce an positional argument list which
+   * is equivalent to the original argumnet list, except the expressions are now
+   * fit for consumption by [[ResolveFunctions]]
+   *
+   * @param expressions The list of positional and named argument expressions
+   * @tparam T The actual expression class.
+   * @return positional argument list
+   */
+  final def getRearrangedExpressions[T <: Expression : ClassTag](
+      expressions: Seq[Expression], functionName: String): Seq[Expression] = {
+
+    if (!expressions.exists(_.isInstanceOf[NamedArgumentExpression])) {
+      return expressions
+    }
+
+    import scala.reflect.runtime.currentMirror
+
+    // This code heavily utilizes Scala reflection which is unfamiliar to most developers.
+    // Here are the steps of this function:
+    // 1. Obtain the module symbol for the companion object of the function expression.
+    // 2. Obtain the module class symbol that represents the companion object.
+    // 3. Check if the base classes of the module class symbol contains SupportsNamedArguments.
+    //    This checks if the companion object is an implementor of SupportsNamedArguments.
+    // 4. Check if the module class symbol is a top level object. Reflection is unable to
+    //    obtain a companion object instance if it is member of some enclosing class unless
+    //    instance of said enclosing class is provided which we do not have.
+    // 5. Use reflection to obtain instance of companion object and perform immediate cast to
+    //    SupportsNamedArguments as it is already verified the cast is safe.
+    // 6. Obtain function signature and rearrange expression according to the given signature.
+    val runtimeClass = scala.reflect.classTag[T].runtimeClass
+    val targetModuleSymbol = currentMirror.classSymbol(runtimeClass).companion
+    val parentClass = scala.reflect.classTag[SupportsNamedArguments].runtimeClass
+    val parentSymbol = currentMirror.classSymbol(parentClass)
+
+    targetModuleSymbol match {
+      case scala.reflect.runtime.universe.NoSymbol =>
+        throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
+      case _ =>
+        val moduleClassSymbol = targetModuleSymbol.asModule.moduleClass.asClass
+        if (moduleClassSymbol.baseClasses.contains(parentSymbol)) {
+          if (currentMirror.runtimeClass(moduleClassSymbol).getEnclosingClass != null) {
+            throw QueryCompilationErrors.cannotObtainCompanionObjectInstance(functionName)
+          }
+          val instance = currentMirror.reflectModule(targetModuleSymbol.asModule)
+            .instance.asInstanceOf[SupportsNamedArguments]

Review Comment:
   how do we know this `asInstanceOf` will succeed?



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -1781,6 +1787,11 @@
       "Not allowed to implement multiple UDF interfaces, UDF class <className>."
     ]
   },
+  "NAMED_ARGUMENTS_NOT_SUPPORTED" : {
+    "message" : [
+      "Named arguments are not supported for function <functionName>; please retry the query with positional arguments to the function call instead."

Review Comment:
   @MaxGekk note that even if the SQL config is enabled, this feature will only work for the subset of SQL functions that have explicitly opted into support for named arguments (by defining the argument names). We probably want to take this fact into account in this error message string as well.



##########
sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql:
##########
@@ -1,5 +1,36 @@
+-- Test for named arguments for Mask
 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-@$#');
+
+-- Test for named arguments for CountMinSketchAgg
+create temporary view t2 as select * from values
+  ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'),
+  ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'),
+  ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null),
+  ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'),
+  ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'),
+  ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'),
+  ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'),
+  ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null)
+  as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i);
+
+SELECT hex(count_min_sketch(t2d, seed => 1, epsilon => 0.5d, confidence => 0.5d)) FROM t2;
+
+-- Unexpected positional argument

Review Comment:
   also add a test with an invalid function call with a mix of positional and named arguments, where one of the positional arguments corresponds to an argument type declaration in the function signature with the same name as one of the provided named arguments? If needed, add a new error class for this case?



##########
sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql:
##########
@@ -1,5 +1,36 @@
+-- Test for named arguments for Mask
 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-@$#');
+
+-- Test for named arguments for CountMinSketchAgg
+create temporary view t2 as select * from values
+  ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'),
+  ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'),
+  ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null),
+  ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'),
+  ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'),
+  ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'),
+  ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'),
+  ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null)
+  as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i);
+
+SELECT hex(count_min_sketch(t2d, seed => 1, epsilon => 0.5d, confidence => 0.5d)) FROM t2;
+
+-- Unexpected positional argument
 SELECT mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd');
+-- Duplicate parameter assignment

Review Comment:
   @ueshin FYI
   We also now support the TABLE keyword for arguments for tbale-valued functions (https://github.com/apache/spark/pull/41750).
   
   Can you add some tests with calls to `mask` with provided arguments of the form `TABLE name` where the `name` may or may not match the expected argument names, to show the behavior? (It may be necessary to sync your PR to pick up this change.)
   
   To reduce the scope of this PR, it may be prudent to just explicitly return an error in the event of any of these type of arguments if the function accepts named args, with a specific error class, until we have the time to look more closely later.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] anchovYu commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "anchovYu (via GitHub)" <gi...@apache.org>.
anchovYu commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1254720472


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * The class which companion objects of function expression may implement to
+ * support named arguments for that function expression.
+ *
+ * Example:
+ *  object CountMinSketchAgg extends SupportsNamedArguments {
+ *    final val functionSignature = FunctionSignature(Seq(
+ *      NamedArgument("column",
+ *          FixedArgumentType(TypeCollection(IntegralType, StringType, BinaryType))),
+ *      NamedArgument("epsilon", FixedArgumentType(DoubleType)),
+ *      NamedArgument("confidence", FixedArgumentType(DoubleType)),
+ *      NamedArgument("seed", FixedArgumentType(IntegerType))
+ *    ))
+ *    override def functionSignatures: Seq[FunctionSignature] = Seq(functionSignature)
+ *  }
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use this provided implementation as it is consistent with
+   * the SQL standard. If absolutely necessary the developer can choose to override the default
+   * behavior for additional flexibility.
+   *
+   * @param expectedSignature Function signature that denotes positional order of arguments
+   * @param providedArguments The sequence of expressions from function invocation
+   * @param functionName The name of the function invoked for debugging purposes
+   * @return positional order of arguments according to FunctionSignature obtained
+   *         by changing the order of the above provided arguments
+   */
+  protected def rearrange(
+      expectedSignature: FunctionSignature,
+      providedArguments: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(expectedSignature, providedArguments, functionName)
+  }
+}
+
+object SupportsNamedArguments {
+
+  /**
+   * Given a generic type, we check if the companion object of said type exists.
+   * If that object extends the trait [[SupportsNamedArguments]], then we rearrange
+   * the expressions in the order specified by the object.
+   *
+   * It is here we resubstitute [[Unevaluable]] [[NamedArgumentExpression]]s with
+   * normal expressions. This method will produce an positional argument list which
+   * is equivalent to the original argumnet list, except the expressions are now
+   * fit for consumption by [[ResolveFunctions]]
+   *
+   * @param expressions The list of positional and named argument expressions
+   * @tparam T The actual expression class.
+   * @return positional argument list
+   */
+  final def getRearrangedExpressions[T <: Expression : ClassTag](
+      expressions: Seq[Expression], functionName: String): Seq[Expression] = {
+
+    if (!expressions.exists(_.isInstanceOf[NamedArgumentExpression])) {
+      return expressions
+    }
+
+    import scala.reflect.runtime.currentMirror
+
+    // This code heavily utilizes Scala reflection which is unfamiliar to most developers.
+    // Here are the steps of this function:
+    // 1. Obtain the module symbol for the companion object of the function expression.
+    // 2. Obtain the module class symbol that represents the companion object.
+    // 3. Check if the base classes of the module class symbol contains SupportsNamedArguments.
+    //    This checks if the companion object is an implementor of SupportsNamedArguments.
+    // 4. Check if the module class symbol is a top level object. Reflection is unable to
+    //    obtain a companion object instance if it is member of some enclosing class unless
+    //    instance of said enclosing class is provided which we do not have.
+    // 5. Use reflection to obtain instance of companion object and perform immediate cast to
+    //    SupportsNamedArguments as it is already verified the cast is safe.
+    // 6. Obtain function signature and rearrange expression according to the given signature.
+    val runtimeClass = scala.reflect.classTag[T].runtimeClass
+    val targetModuleSymbol = currentMirror.classSymbol(runtimeClass).companion
+    val parentClass = scala.reflect.classTag[SupportsNamedArguments].runtimeClass
+    val parentSymbol = currentMirror.classSymbol(parentClass)
+
+    if(targetModuleSymbol == scala.reflect.runtime.universe.NoSymbol) {
+      throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
+    }
+
+    val moduleClassSymbol = targetModuleSymbol.asModule.moduleClass.asClass
+    if (!moduleClassSymbol.baseClasses.contains(parentSymbol)) {
+      throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
+    }
+    if (currentMirror.runtimeClass(moduleClassSymbol).getEnclosingClass != null) {
+      throw QueryCompilationErrors.cannotObtainCompanionObjectInstance(functionName)
+    }
+    val instance = currentMirror.reflectModule(targetModuleSymbol.asModule)
+      .instance.asInstanceOf[SupportsNamedArguments]
+    if (instance.functionSignatures.size != 1) {
+      throw QueryCompilationErrors.multipleFunctionSignatures(
+        functionName, instance.functionSignatures)
+    }
+    instance.rearrange(instance.functionSignatures.head, expressions, functionName)
+  }
+
+  final def defaultRearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters: Seq[NamedArgument] = functionSignature.parameters
+    val firstNamedArgIdx: Int = args.indexWhere(_.isInstanceOf[NamedArgumentExpression])
+    val (positionalArgs, namedArgs) = args.splitAt(firstNamedArgIdx)
+    val namedParameters: Seq[NamedArgument] = parameters.drop(positionalArgs.size)
+
+    // Performing some checking to ensure valid argument list
+    val allParameterNames: Seq[String] = parameters.map(_.name)
+    val parameterNamesSet: Set[String] = allParameterNames.toSet
+    val assignedParameterSet = collection.mutable.Set[String](
+      allParameterNames.take(positionalArgs.size): _*)
+    for (arg <- namedArgs) {
+      arg match {
+        case namedArg: NamedArgumentExpression =>
+          if (assignedParameterSet.contains(namedArg.key)) {
+            throw QueryCompilationErrors.duplicateRoutineParameterAssignment(
+              functionName, namedArg.key)
+          }
+          if (!parameterNamesSet.contains(namedArg.key)) {
+            throw QueryCompilationErrors.unrecognizedParameterName(functionName, namedArg.key,
+              parameterNamesSet.toSeq)
+          }
+          assignedParameterSet.add(namedArg.key)
+        case _ =>
+          throw QueryCompilationErrors.unexpectedPositionalArgument(functionName)
+      }
+    }
+
+    // Construct a map from argument name to value for argument rearrangement
+    val namedArgMap = namedArgs.map { arg =>
+      val namedArg = arg.asInstanceOf[NamedArgumentExpression]
+      namedArg.key -> namedArg.value
+    }.toMap
+
+    // Rearrange named arguments to match their positional order
+    val rearrangedNamedArgs: Seq[Expression] = namedParameters.map { param =>
+      namedArgMap.getOrElse(
+        param.name,
+        if (param.default.isEmpty) {
+          throw QueryCompilationErrors.requiredParameterNotFound(functionName, param.name)
+        } else {
+          param.default.get
+        }
+      )
+    }
+    positionalArgs ++ rearrangedNamedArgs
+  }
+}
+
+/**
+ * Identifies which forms of provided argument values are expected for each call
+ * to the associated SQL function
+ */
+trait NamedArgumentType
+
+/**
+ * Represents a named argument that expects a scalar value of one specific DataType
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * Represents a parameter of a function expression. Function expressions should use this class
+ * to construct the argument lists returned in [[SupportsNamedArguments.functionSignatures]]
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+                          dataType: NamedArgumentType,

Review Comment:
   Could you read this guide https://github.com/databricks/scala-style-guide#indent, go through your PR and confirm that it fully follows the indentation / space specification?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1253748967


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -1781,6 +1787,12 @@
       "Not allowed to implement multiple UDF interfaces, UDF class <className>."
     ]
   },
+  "NAMED_ARGUMENTS_NOT_SUPPORTED" : {
+    "message" : [
+      "Named arguments are not supported for function <functionName>; please retry the query with positional arguments to the function call instead or enable the feature using the SQL config \"spark.sql.allowNamedFunctionArguments\"."

Review Comment:
   Actually, you are right. There is a separate exception for this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1261550835


##########
sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql:
##########
@@ -1,5 +1,36 @@
+-- Test for named arguments for Mask
 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-@$#');
+
+-- Test for named arguments for CountMinSketchAgg
+create temporary view t2 as select * from values
+  ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'),
+  ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'),
+  ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null),
+  ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'),
+  ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'),
+  ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'),
+  ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'),
+  ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null)
+  as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i);
+
+SELECT hex(count_min_sketch(t2d, seed => 1, epsilon => 0.5d, confidence => 0.5d)) FROM t2;
+
+-- Unexpected positional argument
 SELECT mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd');
+-- Duplicate parameter assignment

Review Comment:
   Will add this shortly. I just wanted to confirm with Serge to see if this was a proper way of doing it. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonwang-db commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "allisonwang-db (via GitHub)" <gi...@apache.org>.
allisonwang-db commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1261749089


##########
sql/core/src/test/resources/sql-tests/results/named-function-arguments.sql.out:
##########
@@ -2,121 +2,258 @@
 -- !query
 SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd')
 -- !query schema
+struct<mask(AbCD123-@$#, Q, q, d, o):string>
+-- !query output
+QqQQdddoooo
+
+
+-- !query
+SELECT mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbCD123-@$#')
+-- !query schema
+struct<mask(AbCD123-@$#, Q, q, d, o):string>
+-- !query output
+QqQQdddoooo
+
+
+-- !query
+SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', digitChar => 'd')
+-- !query schema
+struct<mask(AbCD123-@$#, Q, q, d, NULL):string>
+-- !query output
+QqQQddd-@$#
+
+
+-- !query
+SELECT mask(lowerChar => 'q', upperChar => 'Q', digitChar => 'd', str => 'AbCD123-@$#')
+-- !query schema
+struct<mask(AbCD123-@$#, Q, q, d, NULL):string>
+-- !query output
+QqQQddd-@$#
+
+
+-- !query
+create temporary view t2 as select * from values
+  ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'),
+  ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'),
+  ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null),
+  ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'),
+  ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'),
+  ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'),
+  ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'),
+  ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null)
+  as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT hex(count_min_sketch(t2d, seed => 1, epsilon => 0.5d, confidence => 0.5d)) FROM t2
+-- !query schema
+struct<hex(count_min_sketch(t2d, 0.5, 0.5, 1)):string>
+-- !query output
+00000001000000000000000D0000000100000004000000005D8D6AB90000000000000002000000000000000700000000000000010000000000000003
+
+
+-- !query
+SELECT hex(count_min_sketch(seed => 1, epsilon => 0.5d, confidence => 0.5d, column => t2d)) FROM t2
+-- !query schema
+struct<hex(count_min_sketch(t2d, 0.5, 0.5, 1)):string>
+-- !query output
+00000001000000000000000D0000000100000004000000005D8D6AB90000000000000002000000000000000700000000000000010000000000000003
+
+
+-- !query
+SELECT hex(count_min_sketch(t2d, 0.5d, seed => 1, confidence => 0.5d)) FROM t2
+-- !query schema
+struct<hex(count_min_sketch(t2d, 0.5, 0.5, 1)):string>
+-- !query output
+00000001000000000000000D0000000100000004000000005D8D6AB90000000000000002000000000000000700000000000000010000000000000003
+
+
+-- !query
+SELECT * FROM explode(collection => array(1, 2))
+-- !query schema
+struct<col:int>
+-- !query output
+1
+2
+
+
+-- !query
+SELECT * FROM explode_outer(collection => map('a', 1, 'b', 2))
+-- !query schema
+struct<key:string,value:int>
+-- !query output
+a	1
+b	2
+
+
+-- !query
+CREATE OR REPLACE TEMPORARY VIEW v AS SELECT id FROM range(0, 8)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * FROM explode(collection => TABLE v)
+-- !query schema
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT",
+  "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
   "sqlState" : "42K09",
   "messageParameters" : {
-    "inputExpr" : "\"namedargumentexpression(q)\"",
-    "inputName" : "upperChar",
-    "inputType" : "\"STRING\"",
-    "sqlExpr" : "\"mask(AbCD123-@$#, namedargumentexpression(q), namedargumentexpression(Q), namedargumentexpression(o), namedargumentexpression(d))\""
+    "inputSql" : "\"outer(__auto_generated_subquery_name_0.c)\"",
+    "inputType" : "\"STRUCT<id: BIGINT>\"",
+    "paramIndex" : "1",
+    "requiredType" : "(\"ARRAY\" or \"MAP\")",
+    "sqlExpr" : "\"explode(outer(__auto_generated_subquery_name_0.c))\""
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 15,
+    "stopIndex" : 44,
+    "fragment" : "explode(collection => TABLE v)"
+  } ]
+}
+
+
+-- !query
+SELECT mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "UNEXPECTED_POSITIONAL_ARGUMENT",
+  "sqlState" : "4274K",
+  "messageParameters" : {
+    "functionName" : "`mask`"
   },
   "queryContext" : [ {
     "objectType" : "",
     "objectName" : "",
     "startIndex" : 8,
     "stopIndex" : 98,
-    "fragment" : "mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd')"
+    "fragment" : "mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd')"
   } ]
 }
 
 
 -- !query
-SELECT mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbCD123-@$#')
+SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', digitChar => 'e')
 -- !query schema
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT",
-  "sqlState" : "42K09",
+  "errorClass" : "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.DOUBLE_NAMED_ARGUMENT_REFERENCE",
+  "sqlState" : "4274K",
   "messageParameters" : {
-    "inputExpr" : "\"namedargumentexpression(Q)\"",
-    "inputName" : "upperChar",
-    "inputType" : "\"STRING\"",
-    "sqlExpr" : "\"mask(namedargumentexpression(q), namedargumentexpression(Q), namedargumentexpression(o), namedargumentexpression(d), namedargumentexpression(AbCD123-@$#))\""
+    "functionName" : "`mask`",
+    "parameterName" : "`digitChar`"
   },
   "queryContext" : [ {
     "objectType" : "",
     "objectName" : "",
     "startIndex" : 8,
-    "stopIndex" : 105,
-    "fragment" : "mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbCD123-@$#')"
+    "stopIndex" : 116,
+    "fragment" : "mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', digitChar => 'e')"
   } ]
 }
 
 
 -- !query
-SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', digitChar => 'd')
+SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbC')
 -- !query schema
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT",
-  "sqlState" : "42K09",
+  "errorClass" : "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.POSITIONAL_AND_NAMED_ARGUMENT_DOUBLE_REFERENCE",
+  "sqlState" : "4274K",
   "messageParameters" : {
-    "inputExpr" : "\"namedargumentexpression(q)\"",
-    "inputName" : "upperChar",
-    "inputType" : "\"STRING\"",
-    "sqlExpr" : "\"mask(AbCD123-@$#, namedargumentexpression(q), namedargumentexpression(Q), namedargumentexpression(d), NULL)\""
+    "functionName" : "`mask`",
+    "parameterName" : "`str`"
   },
   "queryContext" : [ {
     "objectType" : "",
     "objectName" : "",
     "startIndex" : 8,
-    "stopIndex" : 80,
-    "fragment" : "mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', digitChar => 'd')"
+    "stopIndex" : 112,
+    "fragment" : "mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbC')"
   } ]
 }
 
 
 -- !query
-SELECT mask(lowerChar => 'q', upperChar => 'Q', digitChar => 'd', str => 'AbCD123-@$#')
+SELECT mask(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",
+  "errorClass" : "REQUIRED_PARAMETER_NOT_FOUND",
+  "sqlState" : "4274K",
   "messageParameters" : {
-    "inputExpr" : "\"namedargumentexpression(Q)\"",
-    "inputName" : "upperChar",
-    "inputType" : "\"STRING\"",
-    "sqlExpr" : "\"mask(namedargumentexpression(q), namedargumentexpression(Q), namedargumentexpression(d), namedargumentexpression(AbCD123-@$#), NULL)\""
+    "functionName" : "`mask`",
+    "parameterName" : "`str`"
   },
   "queryContext" : [ {
     "objectType" : "",
     "objectName" : "",
     "startIndex" : 8,
-    "stopIndex" : 87,
-    "fragment" : "mask(lowerChar => 'q', upperChar => 'Q', digitChar => 'd', str => 'AbCD123-@$#')"
+    "stopIndex" : 83,
+    "fragment" : "mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd')"
   } ]
 }
 
 
 -- !query
-SELECT mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd')
+SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', cellular => 'automata')
 -- !query schema
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE",
-  "sqlState" : "42K09",
+  "errorClass" : "UNRECOGNIZED_PARAMETER_NAME",
+  "sqlState" : "4274K",
   "messageParameters" : {
-    "exprName" : "upperChar",
-    "sqlExpr" : "\"mask(namedargumentexpression(q), AbCD123-@$#, namedargumentexpression(Q), namedargumentexpression(o), namedargumentexpression(d))\""
+    "argumentName" : "`cellular`",
+    "functionName" : "`mask`",
+    "proposal" : "`str` `upperChar` `otherChar` "
   },
   "queryContext" : [ {
     "objectType" : "",
     "objectName" : "",
     "startIndex" : 8,
-    "stopIndex" : 98,
-    "fragment" : "mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd')"
+    "stopIndex" : 122,
+    "fragment" : "mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', cellular => 'automata')"
   } ]
 }
+
+
+-- !query
+SELECT encode(str => 'a', charset => 'utf-8')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkException
+{
+  "errorClass" : "INTERNAL_ERROR",
+  "sqlState" : "XX000",
+  "messageParameters" : {
+    "message" : "Cannot generate code for expression: str => a"

Review Comment:
   Is this internal error "cannot generate code for expression" expected?



##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/NamedArgumentFunctionSuite.scala:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.analysis
+
+// import org.apache.spark.SparkThrowable
+// import org.apache.spark.sql.catalyst.InternalRow
+// import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, NamedArgumentExpression}
+// import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
+// import org.apache.spark.sql.catalyst.plans.logical.{FixedArgumentType, FunctionSignature,
+// NamedArgument, SupportsNamedArguments}
+// import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLId
+// import org.apache.spark.sql.types.{DataType, StringType}
+//
+// class NamedArgumentFunctionSuite extends AnalysisTest {

Review Comment:
   Do we need this file?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+object SupportsNamedArguments {
+  final def defaultRearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters: Seq[NamedArgument] = functionSignature.parameters
+    val firstNamedArgIdx: Int = args.indexWhere(_.isInstanceOf[NamedArgumentExpression])
+    val (positionalArgs, namedArgs) =
+      if (firstNamedArgIdx == -1) {
+        (args, Nil)
+      } else {
+        args.splitAt(firstNamedArgIdx)
+      }
+    val namedParameters: Seq[NamedArgument] = parameters.drop(positionalArgs.size)
+
+    // Performing some checking to ensure valid argument list
+    val allParameterNames: Seq[String] = parameters.map(_.name)
+    val parameterNamesSet: Set[String] = allParameterNames.toSet
+    val positionalParametersSet = allParameterNames.take(positionalArgs.size).toSet
+    val namedParametersSet = collection.mutable.Set[String]()
+
+    for (arg <- namedArgs) {
+      arg match {
+        case namedArg: NamedArgumentExpression =>
+          val parameterName = namedArg.key
+          if (!parameterNamesSet.contains(parameterName)) {
+            throw QueryCompilationErrors.unrecognizedParameterName(functionName, namedArg.key,
+              parameterNamesSet.toSeq)
+          }
+          if (positionalParametersSet.contains(parameterName)) {
+            throw QueryCompilationErrors.positionalAndNamedArgumentDoubleReference(
+              functionName, namedArg.key)
+          }
+          if (namedParametersSet.contains(parameterName)) {
+            throw QueryCompilationErrors.doubleNamedArgumentReference(
+              functionName, namedArg.key)
+          }
+          namedParametersSet.add(namedArg.key)
+        case _ =>
+          throw QueryCompilationErrors.unexpectedPositionalArgument(functionName)
+      }
+    }
+
+    // Construct a map from argument name to value for argument rearrangement
+    val namedArgMap = namedArgs.map { arg =>
+      val namedArg = arg.asInstanceOf[NamedArgumentExpression]
+      namedArg.key -> namedArg.value
+    }.toMap
+
+    // Rearrange named arguments to match their positional order
+    val rearrangedNamedArgs: Seq[Expression] = namedParameters.map { param =>
+      namedArgMap.getOrElse(
+        param.name,
+        if (param.default.isEmpty) {
+          throw QueryCompilationErrors.requiredParameterNotFound(functionName, param.name)
+        } else {
+          param.default.get
+        }
+      )
+    }
+    positionalArgs ++ rearrangedNamedArgs
+  }
+}
+
+/**
+ * Identifies which forms of provided argument values are expected for each call
+ * to the associated SQL function
+ */
+trait NamedArgumentType
+
+/**
+ * Represents a named argument that expects a scalar value of one specific DataType
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType

Review Comment:
   Do we still need this?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -1015,6 +1050,62 @@ object TableFunctionRegistry {
   val functionSet: Set[FunctionIdentifier] = builtin.listFunction().toSet
 }
 
-trait ExpressionBuilder {
-  def build(funcName: String, expressions: Seq[Expression]): Expression
+trait Builder[T] {
+  /**
+   * A method that returns the signatures of overloads that is associated with this function
+   *
+   * @return a list of function signatures
+   */
+  def functionSignatures: Option[Seq[FunctionSignature]] = None
+
+  /**
+   * This function rearranges the arguments provided during function invocation in positional order
+   * according to the function signature. This method will fill in the default values if optional
+   * parmaeters do not have their values specified. Any function which supports named arguments
+   * will have this routine invoked, even if no named arguments are present in the argument list.
+   * This is done to eliminate constructor overloads in some methods which use them for default
+   * values prior to the implementation of the named argument framework. This function will also
+   * check if the number of arguments are correct. If that is not the case, then an error will be thrown.
+   *
+   * IMPORTANT: This method will be called before the [[Builder.build]] method is invoked. It is
+   * guaranteed that the expressions provided to the [[Builder.build]] functions forms a valid set
+   * of argument expressions that can be used in the construction of the function expression.
+   *
+   * @param expectedSignature The method signature which we rearrange our arguments according to
+   * @param providedArguments The list of arguments passed from function invocation
+   * @param functionName The name of the function
+   * @return The rearranged arugument list with arguments in positional order
+   */
+  def rearrange(
+      expectedSignature: FunctionSignature,
+      providedArguments: Seq[Expression],
+      functionName: String) : Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(expectedSignature, providedArguments, functionName)
+  }
+
+  def build(funcName: String, expressions: Seq[Expression]): T
+}
+
+/**
+ * A trait used for scalar valued functions that defines how their expression representations
+ * are constructed in [[FunctionRegistry]]
+ */
+trait ExpressionBuilder extends Builder[Expression]
+
+/**
+ * A trait used for table valued functions that defines how their expression representations
+ * are constructed in [[FunctionRegistry]]

Review Comment:
   TableFunctionRegistry



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+object SupportsNamedArguments {
+  final def defaultRearrange(functionSignature: FunctionSignature,

Review Comment:
   Can we add some tests for this function?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala:
##########
@@ -431,6 +432,22 @@ case class Explode(child: Expression) extends ExplodeBase {
     copy(child = newChild)
 }
 
+trait ExplodeGeneratorBuilderBase extends GeneratorBuilder {

Review Comment:
   If we support the named argument for Explode, we should also update its ExpressionDescription and make sure `DESCRIBE FUNCTION` shows the correct information.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala:
##########
@@ -431,6 +432,22 @@ case class Explode(child: Expression) extends ExplodeBase {
     copy(child = newChild)
 }
 
+trait ExplodeGeneratorBuilderBase extends GeneratorBuilder {
+  override def functionSignatures: Option[Seq[FunctionSignature]] =
+    Some(Seq(FunctionSignature(Seq(NamedArgument("collection")))))

Review Comment:
   Does it have to be defined like this? Can this be simplified? 



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -1015,6 +1050,62 @@ object TableFunctionRegistry {
   val functionSet: Set[FunctionIdentifier] = builtin.listFunction().toSet
 }
 
-trait ExpressionBuilder {
-  def build(funcName: String, expressions: Seq[Expression]): Expression
+trait Builder[T] {
+  /**
+   * A method that returns the signatures of overloads that is associated with this function
+   *
+   * @return a list of function signatures
+   */
+  def functionSignatures: Option[Seq[FunctionSignature]] = None
+
+  /**
+   * This function rearranges the arguments provided during function invocation in positional order
+   * according to the function signature. This method will fill in the default values if optional
+   * parmaeters do not have their values specified. Any function which supports named arguments
+   * will have this routine invoked, even if no named arguments are present in the argument list.
+   * This is done to eliminate constructor overloads in some methods which use them for default
+   * values prior to the implementation of the named argument framework. This function will also
+   * check if the number of arguments are correct. If that is not the case, then an error will be thrown.
+   *
+   * IMPORTANT: This method will be called before the [[Builder.build]] method is invoked. It is
+   * guaranteed that the expressions provided to the [[Builder.build]] functions forms a valid set
+   * of argument expressions that can be used in the construction of the function expression.
+   *
+   * @param expectedSignature The method signature which we rearrange our arguments according to
+   * @param providedArguments The list of arguments passed from function invocation
+   * @param functionName The name of the function
+   * @return The rearranged arugument list with arguments in positional order
+   */
+  def rearrange(
+      expectedSignature: FunctionSignature,
+      providedArguments: Seq[Expression],
+      functionName: String) : Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(expectedSignature, providedArguments, functionName)
+  }
+
+  def build(funcName: String, expressions: Seq[Expression]): T
+}
+
+/**
+ * A trait used for scalar valued functions that defines how their expression representations
+ * are constructed in [[FunctionRegistry]]
+ */
+trait ExpressionBuilder extends Builder[Expression]
+
+/**
+ * A trait used for table valued functions that defines how their expression representations
+ * are constructed in [[FunctionRegistry]]
+ */
+trait GeneratorBuilder extends Builder[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

Review Comment:
   If each table-valued generator function needs to extend this GeneratorBuilder, why do we need `funcName` here? 



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -973,6 +994,20 @@ object TableFunctionRegistry {
     (name, (info, (expressions: Seq[Expression]) => builder(expressions)))
   }
 
+  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)
+      val expr = builder.build(name, rearrangedExpressions)
+      expr

Review Comment:
   `expr` -> `plan` this should be a logical plan



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1253690861


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+    dataType: NamedArgumentType,
+    default: Option[Expression] = None)
+
+/**
+ * Represents a method signature and the list of arguments it receives as input.
+ * Currently, overloads are not supported and only one FunctionSignature is allowed
+ * per function expression.
+ *
+ * @param parameters The list of arguments which the function takes
+ */
+case class FunctionSignature(parameters: Seq[NamedArgument])
+
+/**
+ * The class which companion objects of function expression implement to
+ * support named arguments for that function expression.
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use the provided version rearrange as it is consistent with
+   * the SQL standard. If absolutely necessary the developer can choose to override the default
+   * behavior for additional flexibility.
+   *
+   * @param functionSignature Function signature that denotes positional order of arguments
+   * @param args The sequence of expressions from function invocation
+   * @param functionName The name of the function invoked for debugging purposes
+   * @return positional order of arguments according to FunctionSignature
+   */
+  protected def rearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(functionSignature, args, functionName)
+  }
+}
+
+object SupportsNamedArguments {
+
+  /**
+   * Given a generic type, we check if the companion object of said type exists.
+   * If that object extends the trait [[SupportsNamedArguments]], then we rearrange
+   * the expressions in the order specified by the object.
+   *
+   * It is here we resubstitute [[Unevaluable]] [[NamedArgumentExpression]]s with
+   * normal expressions. This method will produce an positional argument list which
+   * is equivalent to the original argumnet list, except the expressions are now
+   * fit for consumption by [[ResolveFunctions]]
+   *
+   * @param expressions The list of positional and named argument expressions
+   * @tparam T The actual expression class.
+   * @return positional argument list
+   */
+  final def getRearrangedExpressions[T <: Expression : ClassTag](
+      expressions: Seq[Expression], functionName: String): Seq[Expression] = {
+
+    if (!expressions.exists(_.isInstanceOf[NamedArgumentExpression])) {
+      return expressions
+    }
+
+    import scala.reflect.runtime.currentMirror
+
+    // This code heavily utilizes Scala reflection which is unfamiliar to most developers.
+    // Here are the steps of this function:
+    // 1. Obtain the module symbol for the companion object of the function expression.
+    // 2. Obtain the module class symbol that represents the companion object.
+    // 3. Check if the base classes of the module class symbol contains SupportsNamedArguments.
+    //    This checks if the companion object is an implementor of SupportsNamedArguments.
+    // 4. Check if the module class symbol is a top level object. Reflection is unable to
+    //    obtain a companion object instance if it is member of some enclosing class unless
+    //    instance of said enclosing class is provided which we do not have.
+    // 5. Use reflection to obtain instance of companion object and perform immediate cast to
+    //    SupportsNamedArguments as it is already verified the cast is safe.
+    // 6. Obtain function signature and rearrange expression according to the given signature.
+    val runtimeClass = scala.reflect.classTag[T].runtimeClass
+    val targetModuleSymbol = currentMirror.classSymbol(runtimeClass).companion
+    val parentClass = scala.reflect.classTag[SupportsNamedArguments].runtimeClass
+    val parentSymbol = currentMirror.classSymbol(parentClass)
+
+    targetModuleSymbol match {
+      case scala.reflect.runtime.universe.NoSymbol =>
+        throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
+      case _ =>
+        val moduleClassSymbol = targetModuleSymbol.asModule.moduleClass.asClass
+        if (moduleClassSymbol.baseClasses.contains(parentSymbol)) {
+          if (currentMirror.runtimeClass(moduleClassSymbol).getEnclosingClass != null) {
+            throw QueryCompilationErrors.cannotObtainCompanionObjectInstance(functionName)
+          }
+          val instance = currentMirror.reflectModule(targetModuleSymbol.asModule)
+            .instance.asInstanceOf[SupportsNamedArguments]

Review Comment:
   This is safe since earlier we do the check  ```moduleClassSymbol.baseClasses.contains(parentSymbol)``` . This line checks if any of the classes we inherit includes the SupportNamedArguments trait.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on PR #41864:
URL: https://github.com/apache/spark/pull/41864#issuecomment-1624114319

   @anchovYu Corrected indentation plus added some extra TVF named arguments support.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1262238391


##########
sql/core/src/test/scala/org/apache/spark/sql/CountMinSketchAggQuerySuite.scala:
##########
@@ -61,4 +61,23 @@ class CountMinSketchAggQuerySuite extends QueryTest with SharedSparkSession {
 
     assert(sketch == reference)
   }
+
+  test("count-min sketch with named-arguments") {

Review Comment:
   nit: we can remove this end-to-end test if `named-function-arguments.sql` includes `count_min_sketch`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #41864:
URL: https://github.com/apache/spark/pull/41864#issuecomment-1635471076

   seems `ExpressionInfoSuite` has test failures.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1262846230


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+object SupportsNamedArguments {
+  final def defaultRearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters: Seq[NamedArgument] = functionSignature.parameters
+    val firstNamedArgIdx: Int = args.indexWhere(_.isInstanceOf[NamedArgumentExpression])
+    val (positionalArgs, namedArgs) =
+      if (firstNamedArgIdx == -1) {
+        (args, Nil)

Review Comment:
   OK, we can leave this case as-is for now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1262236718


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala:
##########
@@ -321,3 +322,21 @@ object Mask {
     }
   }
 }
+
+object MaskExpressionBuilder extends ExpressionBuilder {
+  override def functionSignatures: Option[Seq[FunctionSignature]] = {
+    val strArg = NamedArgument("str")
+    val upperCharArg = NamedArgument("upperChar", Some(Literal(Mask.MASKED_UPPERCASE)))
+    val lowerCharArg = NamedArgument("lowerChar", Some(Literal(Mask.MASKED_LOWERCASE)))
+    val digitCharArg = NamedArgument("digitChar", Some(Literal(Mask.MASKED_DIGIT)))
+    val otherCharArg = NamedArgument("otherChar", Some(Literal(Mask.MASKED_IGNORE)))
+    val functionSignature: FunctionSignature = FunctionSignature(Seq(
+      strArg, upperCharArg, lowerCharArg, digitCharArg, otherCharArg))
+    Some(Seq(functionSignature))
+  }
+
+  override def build(funcName: String, expressions: Seq[Expression]): Expression = {

Review Comment:
   shall we check the number of arguments as we did in https://github.com/apache/spark/pull/41864/files#diff-66a6cfe45ec7e9e6be23be40f8ccedf9ec84887abfa6e7d3b9c34ea85bf85e91R222 ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1262766997


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -50,6 +50,90 @@ import org.apache.spark.sql.types._
  */
 private[sql] object QueryCompilationErrors extends QueryErrorsBase {
 
+  def unexpectedRequiredParameterInFunctionSignature(
+      functionName: String, functionSignature: FunctionSignature) : Throwable = {
+    val errorMessage = s"Function $functionName has an unexpected required argument for" +
+      s" the provided function signature $functionSignature. All required arguments should" +
+      s" come before optional arguments."

Review Comment:
   s is not needed.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -50,6 +50,90 @@ import org.apache.spark.sql.types._
  */
 private[sql] object QueryCompilationErrors extends QueryErrorsBase {
 
+  def unexpectedRequiredParameterInFunctionSignature(
+      functionName: String, functionSignature: FunctionSignature) : Throwable = {
+    val errorMessage = s"Function $functionName has an unexpected required argument for" +
+      s" the provided function signature $functionSignature. All required arguments should" +
+      s" come before optional arguments."
+    SparkException.internalError(errorMessage)
+  }
+
+  def multipleFunctionSignatures(functionName: String,
+      functionSignatures: Seq[FunctionSignature]): Throwable = {
+    var errorMessage = s"Function $functionName cannot have multiple method signatures." +
+      s" The function signatures found were: \n"
+    for (functionSignature <- functionSignatures) {
+      errorMessage = errorMessage + s"${functionSignature}\n"
+    }
+    SparkException.internalError(errorMessage)
+  }
+
+  def namedArgumentsNotSupported(functionName: String) : Throwable = {
+    new AnalysisException(
+      errorClass = "NAMED_ARGUMENTS_NOT_SUPPORTED",
+      messageParameters = Map("functionName" -> toSQLId(functionName))
+    )
+  }
+
+  def positionalAndNamedArgumentDoubleReference(
+      functionName: String, parameterName: String) : Throwable = {
+    val errorClass =
+      "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.POSITIONAL_AND_NAMED_ARGUMENT_DOUBLE_REFERENCE"

Review Comment:
   It is not long. Is it possible to make it shorter?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -50,6 +50,90 @@ import org.apache.spark.sql.types._
  */
 private[sql] object QueryCompilationErrors extends QueryErrorsBase {
 
+  def unexpectedRequiredParameterInFunctionSignature(
+      functionName: String, functionSignature: FunctionSignature) : Throwable = {
+    val errorMessage = s"Function $functionName has an unexpected required argument for" +
+      s" the provided function signature $functionSignature. All required arguments should" +
+      s" come before optional arguments."
+    SparkException.internalError(errorMessage)
+  }
+
+  def multipleFunctionSignatures(functionName: String,
+      functionSignatures: Seq[FunctionSignature]): Throwable = {
+    var errorMessage = s"Function $functionName cannot have multiple method signatures." +
+      s" The function signatures found were: \n"

Review Comment:
   s is not needed.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -50,6 +50,90 @@ import org.apache.spark.sql.types._
  */
 private[sql] object QueryCompilationErrors extends QueryErrorsBase {
 
+  def unexpectedRequiredParameterInFunctionSignature(
+      functionName: String, functionSignature: FunctionSignature) : Throwable = {
+    val errorMessage = s"Function $functionName has an unexpected required argument for" +
+      s" the provided function signature $functionSignature. All required arguments should" +
+      s" come before optional arguments."
+    SparkException.internalError(errorMessage)
+  }
+
+  def multipleFunctionSignatures(functionName: String,
+      functionSignatures: Seq[FunctionSignature]): Throwable = {
+    var errorMessage = s"Function $functionName cannot have multiple method signatures." +
+      s" The function signatures found were: \n"
+    for (functionSignature <- functionSignatures) {
+      errorMessage = errorMessage + s"${functionSignature}\n"
+    }
+    SparkException.internalError(errorMessage)
+  }
+
+  def namedArgumentsNotSupported(functionName: String) : Throwable = {
+    new AnalysisException(
+      errorClass = "NAMED_ARGUMENTS_NOT_SUPPORTED",
+      messageParameters = Map("functionName" -> toSQLId(functionName))
+    )
+  }
+
+  def positionalAndNamedArgumentDoubleReference(
+      functionName: String, parameterName: String) : Throwable = {
+    val errorClass =
+      "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.POSITIONAL_AND_NAMED_ARGUMENT_DOUBLE_REFERENCE"
+    new AnalysisException(
+      errorClass = errorClass,
+      messageParameters = Map(
+        "functionName" -> toSQLId(functionName),
+        "parameterName" -> toSQLId(parameterName))
+    )
+  }
+
+  def doubleNamedArgumentReference(
+      functionName: String, parameterName: String): Throwable = {
+    val errorClass =
+      "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.DOUBLE_NAMED_ARGUMENT_REFERENCE"
+    new AnalysisException(
+      errorClass = errorClass,
+      messageParameters = Map(
+        "functionName" -> toSQLId(functionName),
+        "parameterName" -> toSQLId(parameterName))
+    )
+  }
+
+  def requiredParameterNotFound(
+      functionName: String, parameterName: String) : Throwable = {
+    new AnalysisException(
+      errorClass = "REQUIRED_PARAMETER_NOT_FOUND",
+      messageParameters = Map(
+        "functionName" -> toSQLId(functionName),
+        "parameterName" -> toSQLId(parameterName))
+    )
+  }
+
+  def unrecognizedParameterName(
+      functionName: String, argumentName: String, candidates: Seq[String]): Throwable = {
+    import org.apache.spark.sql.catalyst.util.StringUtils.orderSuggestedIdentifiersBySimilarity
+
+    val inputs = candidates.map(candidate => Seq(candidate)).toSeq
+    val recommendations = orderSuggestedIdentifiersBySimilarity(argumentName, inputs)
+      .take(3)
+    var candidatesString = ""
+    recommendations.foreach(candidatesString += _ + " ")

Review Comment:
   Could just use `mkString`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1261550835


##########
sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql:
##########
@@ -1,5 +1,36 @@
+-- Test for named arguments for Mask
 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-@$#');
+
+-- Test for named arguments for CountMinSketchAgg
+create temporary view t2 as select * from values
+  ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'),
+  ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'),
+  ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null),
+  ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'),
+  ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'),
+  ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'),
+  ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'),
+  ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null)
+  as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i);
+
+SELECT hex(count_min_sketch(t2d, seed => 1, epsilon => 0.5d, confidence => 0.5d)) FROM t2;
+
+-- Unexpected positional argument
 SELECT mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd');
+-- Duplicate parameter assignment

Review Comment:
   Will add this shortly. I just wanted to confirm with Serge to see if this was a proper way of doing it. 



##########
sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql:
##########
@@ -1,5 +1,36 @@
+-- Test for named arguments for Mask
 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-@$#');
+
+-- Test for named arguments for CountMinSketchAgg
+create temporary view t2 as select * from values
+  ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'),
+  ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'),
+  ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null),
+  ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'),
+  ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'),
+  ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'),
+  ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'),
+  ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null)
+  as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i);
+
+SELECT hex(count_min_sketch(t2d, seed => 1, epsilon => 0.5d, confidence => 0.5d)) FROM t2;
+
+-- Unexpected positional argument

Review Comment:
   Will add this shortly. I just wanted to confirm with Serge to see if this was a proper way of doing it. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] anchovYu commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "anchovYu (via GitHub)" <gi...@apache.org>.
anchovYu commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1263066825


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala:
##########
@@ -321,3 +322,21 @@ object Mask {
     }
   }
 }
+
+object MaskExpressionBuilder extends ExpressionBuilder {
+  override def functionSignature: Option[FunctionSignature] = {
+    val strArg = NamedArgument("str")
+    val upperCharArg = NamedArgument("upperChar", Some(Literal(Mask.MASKED_UPPERCASE)))
+    val lowerCharArg = NamedArgument("lowerChar", Some(Literal(Mask.MASKED_LOWERCASE)))
+    val digitCharArg = NamedArgument("digitChar", Some(Literal(Mask.MASKED_DIGIT)))
+    val otherCharArg = NamedArgument("otherChar", Some(Literal(Mask.MASKED_IGNORE)))

Review Comment:
   Existing constructors on mask adds a type information: `Literal(Mask.MASKED_IGNORE, StringType))`. Could you change it?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/FunctionBuilderBase.scala:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+trait FunctionBuilderBase[T] {
+  /**
+   * A method that returns the signatures of overloads that are associated with this function.
+   * Each function signature includes a list of parameters to which the analyzer can
+   * compare a function call with provided arguments to determine if that function
+   * call is a match for the function signature.
+   *
+   * @return a list of function signatures
+   */
+  def functionSignatures: Option[Seq[FunctionSignature]] = None
+
+  /**
+   * This function rearranges the arguments provided during function invocation in positional order
+   * according to the function signature. This method will fill in the default values if optional
+   * parameters do not have their values specified. Any function which supports named arguments
+   * will have this routine invoked, even if no named arguments are present in the argument list.
+   * This is done to eliminate constructor overloads in some methods which use them for default
+   * values prior to the implementation of the named argument framework. This function will also
+   * check if the number of arguments are correct. If that is not the case, then an error will be
+   * thrown.
+   *
+   * IMPORTANT: This method will be called before the [[FunctionBuilderBase.build]] method is
+   * invoked. It is guaranteed that the expressions provided to the [[FunctionBuilderBase.build]]
+   * functions forms a valid set of argument expressions that can be used in the construction of
+   * the function expression.
+   *
+   * @param expectedSignature The method signature which we rearrange our arguments according to
+   * @param providedArguments The list of arguments passed from function invocation
+   * @param functionName The name of the function
+   * @return The rearranged argument list with arguments in positional order
+   */
+  def rearrange(
+                 expectedSignature: FunctionSignature,
+                 providedArguments: Seq[Expression],
+                 functionName: String) : Seq[Expression] = {
+    NamedArgumentsSupport.defaultRearrange(expectedSignature, providedArguments, functionName)
+  }
+
+  def build(funcName: String, expressions: Seq[Expression]): T
+}
+
+object NamedArgumentsSupport {
+  /**
+   * This method is the default routine which rearranges the arguments in positional order according
+   * to the function signature provided. This will also fill in any default values that exists for
+   * optional arguments. This method will also be invoked even if there are no named arguments in
+   * the argument list.
+   *
+   * @param functionSignature The function signature that defines the positional ordering
+   * @param args The argument list provided in function invocation
+   * @param functionName The name of the function
+   * @return A list of arguments rearranged in positional order defined by the provided signature
+   */
+  final def defaultRearrange(
+      functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters: Seq[NamedArgument] = functionSignature.parameters
+    if (parameters.dropWhile(_.default.isEmpty).exists(_.default.isEmpty)) {
+      throw QueryCompilationErrors.unexpectedRequiredParameterInFunctionSignature(
+        functionName, functionSignature)
+    }
+
+    val (positionalArgs, namedArgs) = args.span(!_.isInstanceOf[NamedArgumentExpression])
+    val namedParameters: Seq[NamedArgument] = parameters.drop(positionalArgs.size)
+
+    // The following loop checks for the following:
+    // 1. Unrecognized parameter names
+    // 2. Duplicate routine parameter assignments
+    val allParameterNames: Seq[String] = parameters.map(_.name)
+    val parameterNamesSet: Set[String] = allParameterNames.toSet
+    val positionalParametersSet = allParameterNames.take(positionalArgs.size).toSet
+    val namedParametersSet = collection.mutable.Set[String]()
+
+    for (arg <- namedArgs) {
+      arg match {
+        case namedArg: NamedArgumentExpression =>
+          val parameterName = namedArg.key
+          if (!parameterNamesSet.contains(parameterName)) {
+            throw QueryCompilationErrors.unrecognizedParameterName(functionName, namedArg.key,
+              parameterNamesSet.toSeq)
+          }
+          if (positionalParametersSet.contains(parameterName)) {
+            throw QueryCompilationErrors.positionalAndNamedArgumentDoubleReference(
+              functionName, namedArg.key)
+          }
+          if (namedParametersSet.contains(parameterName)) {
+            throw QueryCompilationErrors.doubleNamedArgumentReference(
+              functionName, namedArg.key)
+          }
+          namedParametersSet.add(namedArg.key)
+        case _ =>
+          throw QueryCompilationErrors.unexpectedPositionalArgument(functionName)
+      }
+    }
+
+    // Check argument list size against provided parameter list length
+    if (parameters.size < args.length) {

Review Comment:
   Thanks. Could you add one for mask in the bunch of error cases in the e2e tests?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1258792210


##########
sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql:
##########
@@ -1,5 +1,36 @@
+-- Test for named arguments for Mask
 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-@$#');
+
+-- Test for named arguments for CountMinSketchAgg
+create temporary view t2 as select * from values
+  ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'),
+  ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'),
+  ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null),
+  ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'),
+  ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'),
+  ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'),
+  ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'),
+  ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null)
+  as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i);
+
+SELECT hex(count_min_sketch(t2d, seed => 1, epsilon => 0.5d, confidence => 0.5d)) FROM t2;
+
+-- Unexpected positional argument
 SELECT mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd');
+-- Duplicate parameter assignment

Review Comment:
   Where is the testing for this (to pass the TABLE keyword for functions that accept named arguments, to show the resulting error messages)?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1258941506


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala:
##########
@@ -344,4 +346,13 @@ object Mask extends SupportsNamedArguments {
       strArg, upperCharArg, lowerCharArg, digitCharArg, otherCharArg))
     Seq(functionSignature)
   }
+
+  override def build(funcName: String, expressions: Seq[Expression]): Expression = {
+    if (expressions.length < 1 || expressions.length > 5) {
+      throw QueryCompilationErrors.wrongNumArgsError(
+        funcName, Seq(1, 2, 3, 4, 5), expressions.length)
+    }
+    new Mask(expressions(0), expressions(1), expressions(2), expressions(3), expressions(4))

Review Comment:
   @anchovYu Thanks for the comments and the thoughts! About this: you are right in that if we want to avoid reflection entirely, we will not be using the ```FunctionRegistryBase.build``` method since that heavily utilizes reflection to find the constructor we want for building the function expression. Therefore, it is true that in cases where we only provide positional arguments that the old version of ```rearrange``` did not handle this case. 
   
   After some thoughts however, I realized that we can implement default arguments as well in a cleaner fashion that what we had with the old way where we would have multiple constructors defined for a single function expression. The ```rearrange``` method would now also be responsible for filling in default values for a function invocation where only positional arguments are provided and if some optional parameter values are left unspecified. In another comment, you were wondering why I checked if the index was -1 when splitting the argument list into positional and named arguments, that check was done precisely to allow the processing of default values for parameters in the case where only positional arguments are provided. I thought this was a fairly elegant way of doing it, although it can be quite confusing. I will definitely need to clean up the code/add some comments to make this more clear.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] anchovYu commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "anchovYu (via GitHub)" <gi...@apache.org>.
anchovYu commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1254714134


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -674,6 +674,12 @@
     ],
     "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>."
+    ],
+    "sqlState" : "4274K"

Review Comment:
   Thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1259022057


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala:
##########
@@ -208,3 +209,20 @@ case class CountMinSketchAgg(
       confidenceExpression = third,
       seedExpression = fourth)
 }
+
+object CountMinSketchAggExpressionBuilder extends ExpressionBuilder {
+  final val functionSignature = FunctionSignature(Seq(
+    NamedArgument("column",
+                  FixedArgumentType(TypeCollection(IntegralType, StringType, BinaryType))),
+    NamedArgument("epsilon", FixedArgumentType(DoubleType)),
+    NamedArgument("confidence", FixedArgumentType(DoubleType)),
+    NamedArgument("seed", FixedArgumentType(IntegerType))

Review Comment:
   We need to make a decision here. Before this PR, SQL functions define the required input types in the corresponding `Expressions`. Now we want to replace it with a new framework. This is fine but we need to make sure the new framework can fully cover the old one.
   
   The old framework is actually quite flexible, you can do arbitrary checks in `Expression.checkInputDataTypes`, or use some convenient trait like `ExpectsInputTypes`. The old framework can also connect to the type coercion framework. For example, the `ImplicitCastInputTypes` trait.
   
   The `PercentileBase` expression is an example. It requires the `percentage` input to be foldable, double-type, and within the range [0, 1.0]. Can the new framework cover it?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on PR #41864:
URL: https://github.com/apache/spark/pull/41864#issuecomment-1636915338

   Migrating to #42020 due to some problems with branches.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1262239659


##########
sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out:
##########
@@ -327,13 +327,14 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
-  "sqlState" : "42605",
+  "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",

Review Comment:
   I feel the previous error is better. Shall we tune the named-parameter framework so that it always checks the num args first?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonwang-db commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "allisonwang-db (via GitHub)" <gi...@apache.org>.
allisonwang-db commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1262805881


##########
sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out:
##########
@@ -327,13 +327,14 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
-  "sqlState" : "42605",
+  "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+  "sqlState" : "42K09",
   "messageParameters" : {
-    "actualNum" : "2",
-    "docroot" : "https://spark.apache.org/docs/latest",
-    "expectedNum" : "1",
-    "functionName" : "`explode`"
+    "inputSql" : "\"1\"",
+    "inputType" : "\"INT\"",
+    "paramIndex" : "1",
+    "requiredType" : "(\"ARRAY\" or \"MAP\")",
+    "sqlExpr" : "\"explode(1)\""
   },
   "queryContext" : [ {
     "objectType" : "",

Review Comment:
   Can we add more named argument tests in this file (table-valued-functions.sql) when the function invocation throws exceptions? For example use named arguments for these tests:
   https://github.com/apache/spark/blob/efed39516c0c4e9654aec447ce91676026368384/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out#L349-L355
   
   I'd like to make sure the exceptions stay the same.



##########
sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out:
##########
@@ -327,13 +327,14 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
-  "sqlState" : "42605",
+  "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+  "sqlState" : "42K09",
   "messageParameters" : {
-    "actualNum" : "2",
-    "docroot" : "https://spark.apache.org/docs/latest",
-    "expectedNum" : "1",
-    "functionName" : "`explode`"
+    "inputSql" : "\"1\"",
+    "inputType" : "\"INT\"",
+    "paramIndex" : "1",
+    "requiredType" : "(\"ARRAY\" or \"MAP\")",
+    "sqlExpr" : "\"explode(1)\""

Review Comment:
   shouldn't this be explode (1,2)?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1263019012


##########
sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql:
##########
@@ -1,5 +1,50 @@
+-- Test for named arguments for Mask

Review Comment:
   I think the current mask-expression.sql test already handles this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] anchovYu commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "anchovYu (via GitHub)" <gi...@apache.org>.
anchovYu commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1261503016


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala:
##########
@@ -344,4 +346,13 @@ object Mask extends SupportsNamedArguments {
       strArg, upperCharArg, lowerCharArg, digitCharArg, otherCharArg))
     Seq(functionSignature)
   }
+
+  override def build(funcName: String, expressions: Seq[Expression]): Expression = {
+    if (expressions.length < 1 || expressions.length > 5) {
+      throw QueryCompilationErrors.wrongNumArgsError(
+        funcName, Seq(1, 2, 3, 4, 5), expressions.length)
+    }
+    new Mask(expressions(0), expressions(1), expressions(2), expressions(3), expressions(4))

Review Comment:
   Have discussed with multiple people, I vote for "also be responsible for filling in default values for a function invocation where only positional arguments are provided and if some optional parameter values are left unspecified". We will enforce the required parameters defined before optional ones.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1259022421


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * The class which companion objects of function expression may implement to
+ * support named arguments for that function expression. Please note that variadic final
+ * arguments are NOT supported for named arguments. Do not use for functions that
+ * has variadic final arguments!
+ *
+ * Example:
+ *  object CountMinSketchAgg extends SupportsNamedArguments {
+ *    final val functionSignature = FunctionSignature(Seq(
+ *      NamedArgument("column",
+ *          FixedArgumentType(TypeCollection(IntegralType, StringType, BinaryType))),
+ *      NamedArgument("epsilon", FixedArgumentType(DoubleType)),
+ *      NamedArgument("confidence", FixedArgumentType(DoubleType)),
+ *      NamedArgument("seed", FixedArgumentType(IntegerType))
+ *    ))
+ *    override def functionSignatures: Seq[FunctionSignature] = Seq(functionSignature)
+ *  }
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use this provided implementation as it is consistent with
+   * the SQL standard. If absolutely necessary the developer can choose to override the default
+   * behavior for additional flexibility.
+   *
+   * @param expectedSignature Function signature that denotes positional order of arguments
+   * @param providedArguments The sequence of expressions from function invocation
+   * @param functionName The name of the function invoked for debugging purposes
+   * @return positional order of arguments according to FunctionSignature obtained
+   *         by changing the order of the above provided arguments
+   */
+  protected def rearrange(
+      expectedSignature: FunctionSignature,
+      providedArguments: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(expectedSignature, providedArguments, functionName)
+  }
+}
+
+object SupportsNamedArguments {
+  final def defaultRearrange(functionSignature: FunctionSignature,

Review Comment:
   looking at this function, seems it doesn't care about the required data types of the arguments?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1262230528


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -887,13 +887,46 @@ 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 <: Builder[_]](
+      name: String,
+      builder: T,
+      expressions: Seq[Expression]) : Seq[Expression] = {
+    val rearrangedExpressions = if (!builder.functionSignatures.isEmpty) {
+      val functionSignatures = builder.functionSignatures.get
+      if (functionSignatures.length != 1) {

Review Comment:
   If we don't have a plan to support function overload soon, shall we restrict it at compile time? `def functionSignatures: Option[Seq[FunctionSignature]] = None` can be `def functionSignatures: Option[FunctionSignature] = None`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1260455152


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala:
##########
@@ -208,3 +209,20 @@ case class CountMinSketchAgg(
       confidenceExpression = third,
       seedExpression = fourth)
 }
+
+object CountMinSketchAggExpressionBuilder extends ExpressionBuilder {
+  final val functionSignature = FunctionSignature(Seq(
+    NamedArgument("column",
+                  FixedArgumentType(TypeCollection(IntegralType, StringType, BinaryType))),
+    NamedArgument("epsilon", FixedArgumentType(DoubleType)),
+    NamedArgument("confidence", FixedArgumentType(DoubleType)),
+    NamedArgument("seed", FixedArgumentType(IntegerType))

Review Comment:
   It's good to make the new framework more capable, but we need to have a clear vision for the eventual system. If eventually we only want the new named-parameter framework to reorder the function parameters, and leave the type check to the existing `Expression` framework, then the named-parameter framework should not define the type information. If we want to move everything to this new framework, we should design it well, by looking at existing functions and define how everything should be done with the new framework.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1262246806


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+object SupportsNamedArguments {
+  /**
+   * This method is the default routine which rearranges the arguments in positional order according
+   * to the function signature provided. This will also fill in any default values that exists for
+   * optional arguments. This method will also be invoked even if there are no named arguments in
+   * the argument list.
+   *
+   * @param functionSignature The function signature that defines the positional ordering
+   * @param args The argument list provided in function invocation
+   * @param functionName The name of the function
+   * @return A list of arguments rearranged in positional order defined by the provided signature
+   */
+  final def defaultRearrange(
+      functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters: Seq[NamedArgument] = functionSignature.parameters
+    val firstOptionalParamIndex: Int = parameters.indexWhere(_.default.isDefined)
+    if (firstOptionalParamIndex != -1 &&
+        parameters.drop(firstOptionalParamIndex).exists(_.default.isEmpty)) {
+      throw QueryCompilationErrors.unexpectedRequiredParameterInFunctionSignature(
+        functionName, functionSignature)
+    }
+
+    val firstNamedArgIdx: Int = args.indexWhere(_.isInstanceOf[NamedArgumentExpression])

Review Comment:
   nit: `val (posArgs, namedArgs) = args.span(!_.isInstanceOf[NamedArgumentExpression])`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1262895833


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -935,9 +969,9 @@ 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)

Review Comment:
   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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1253708557


##########
sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql:
##########
@@ -1,5 +1,36 @@
+-- Test for named arguments for Mask
 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-@$#');
+
+-- Test for named arguments for CountMinSketchAgg
+create temporary view t2 as select * from values
+  ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'),
+  ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'),
+  ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null),
+  ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'),
+  ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'),
+  ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'),
+  ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'),
+  ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null)
+  as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i);
+
+SELECT hex(count_min_sketch(t2d, seed => 1, epsilon => 0.5d, confidence => 0.5d)) FROM t2;
+
+-- Unexpected positional argument

Review Comment:
   I just wanted to clarify: is this covered by DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT? This sounds like a variable was assigned twice. It's just that it's not necessarily referred to by name both times.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on PR #41864:
URL: https://github.com/apache/spark/pull/41864#issuecomment-1622615687

   @dtenedor Addressed comments, let me know what you think!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on PR #41864:
URL: https://github.com/apache/spark/pull/41864#issuecomment-1622543139

   @MaxGekk Comments addressed!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1253689841


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+    dataType: NamedArgumentType,
+    default: Option[Expression] = None)
+
+/**
+ * Represents a method signature and the list of arguments it receives as input.
+ * Currently, overloads are not supported and only one FunctionSignature is allowed
+ * per function expression.
+ *
+ * @param parameters The list of arguments which the function takes
+ */
+case class FunctionSignature(parameters: Seq[NamedArgument])
+
+/**
+ * The class which companion objects of function expression implement to
+ * support named arguments for that function expression.
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use the provided version rearrange as it is consistent with
+   * the SQL standard. If absolutely necessary the developer can choose to override the default
+   * behavior for additional flexibility.
+   *
+   * @param functionSignature Function signature that denotes positional order of arguments
+   * @param args The sequence of expressions from function invocation
+   * @param functionName The name of the function invoked for debugging purposes
+   * @return positional order of arguments according to FunctionSignature
+   */
+  protected def rearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(functionSignature, args, functionName)
+  }
+}
+
+object SupportsNamedArguments {
+
+  /**
+   * Given a generic type, we check if the companion object of said type exists.
+   * If that object extends the trait [[SupportsNamedArguments]], then we rearrange
+   * the expressions in the order specified by the object.
+   *
+   * It is here we resubstitute [[Unevaluable]] [[NamedArgumentExpression]]s with
+   * normal expressions. This method will produce an positional argument list which
+   * is equivalent to the original argumnet list, except the expressions are now
+   * fit for consumption by [[ResolveFunctions]]
+   *
+   * @param expressions The list of positional and named argument expressions
+   * @tparam T The actual expression class.
+   * @return positional argument list
+   */
+  final def getRearrangedExpressions[T <: Expression : ClassTag](
+      expressions: Seq[Expression], functionName: String): Seq[Expression] = {
+
+    if (!expressions.exists(_.isInstanceOf[NamedArgumentExpression])) {
+      return expressions
+    }
+
+    import scala.reflect.runtime.currentMirror
+
+    // This code heavily utilizes Scala reflection which is unfamiliar to most developers.
+    // Here are the steps of this function:
+    // 1. Obtain the module symbol for the companion object of the function expression.
+    // 2. Obtain the module class symbol that represents the companion object.
+    // 3. Check if the base classes of the module class symbol contains SupportsNamedArguments.
+    //    This checks if the companion object is an implementor of SupportsNamedArguments.
+    // 4. Check if the module class symbol is a top level object. Reflection is unable to
+    //    obtain a companion object instance if it is member of some enclosing class unless
+    //    instance of said enclosing class is provided which we do not have.
+    // 5. Use reflection to obtain instance of companion object and perform immediate cast to
+    //    SupportsNamedArguments as it is already verified the cast is safe.
+    // 6. Obtain function signature and rearrange expression according to the given signature.
+    val runtimeClass = scala.reflect.classTag[T].runtimeClass
+    val targetModuleSymbol = currentMirror.classSymbol(runtimeClass).companion
+    val parentClass = scala.reflect.classTag[SupportsNamedArguments].runtimeClass
+    val parentSymbol = currentMirror.classSymbol(parentClass)
+
+    targetModuleSymbol match {
+      case scala.reflect.runtime.universe.NoSymbol =>

Review Comment:
   This syntax I'm afraid is illegal since NoSymbol isn't a type, its a singleton object. That's why I went with the case syntax.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1264026224


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2445,6 +2481,12 @@
     ],
     "sqlState" : "428C4"
   },
+  "UNRECOGNIZED_PARAMETER_NAME" : {

Review Comment:
   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. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1253637205


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2333,6 +2354,11 @@
     ],
     "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."

Review Comment:
   Sounds good! Added the feature.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1253714805


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+    dataType: NamedArgumentType,
+    default: Option[Expression] = None)
+
+/**
+ * Represents a method signature and the list of arguments it receives as input.
+ * Currently, overloads are not supported and only one FunctionSignature is allowed
+ * per function expression.
+ *
+ * @param parameters The list of arguments which the function takes
+ */
+case class FunctionSignature(parameters: Seq[NamedArgument])
+
+/**
+ * The class which companion objects of function expression implement to
+ * support named arguments for that function expression.
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use the provided version rearrange as it is consistent with
+   * the SQL standard. If absolutely necessary the developer can choose to override the default
+   * behavior for additional flexibility.
+   *
+   * @param functionSignature Function signature that denotes positional order of arguments
+   * @param args The sequence of expressions from function invocation
+   * @param functionName The name of the function invoked for debugging purposes
+   * @return positional order of arguments according to FunctionSignature
+   */
+  protected def rearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(functionSignature, args, functionName)
+  }
+}
+
+object SupportsNamedArguments {
+
+  /**
+   * Given a generic type, we check if the companion object of said type exists.
+   * If that object extends the trait [[SupportsNamedArguments]], then we rearrange
+   * the expressions in the order specified by the object.
+   *
+   * It is here we resubstitute [[Unevaluable]] [[NamedArgumentExpression]]s with
+   * normal expressions. This method will produce an positional argument list which
+   * is equivalent to the original argumnet list, except the expressions are now
+   * fit for consumption by [[ResolveFunctions]]
+   *
+   * @param expressions The list of positional and named argument expressions
+   * @tparam T The actual expression class.
+   * @return positional argument list
+   */
+  final def getRearrangedExpressions[T <: Expression : ClassTag](
+      expressions: Seq[Expression], functionName: String): Seq[Expression] = {
+
+    if (!expressions.exists(_.isInstanceOf[NamedArgumentExpression])) {
+      return expressions
+    }
+
+    import scala.reflect.runtime.currentMirror
+
+    // This code heavily utilizes Scala reflection which is unfamiliar to most developers.
+    // Here are the steps of this function:
+    // 1. Obtain the module symbol for the companion object of the function expression.
+    // 2. Obtain the module class symbol that represents the companion object.
+    // 3. Check if the base classes of the module class symbol contains SupportsNamedArguments.
+    //    This checks if the companion object is an implementor of SupportsNamedArguments.
+    // 4. Check if the module class symbol is a top level object. Reflection is unable to
+    //    obtain a companion object instance if it is member of some enclosing class unless
+    //    instance of said enclosing class is provided which we do not have.
+    // 5. Use reflection to obtain instance of companion object and perform immediate cast to
+    //    SupportsNamedArguments as it is already verified the cast is safe.
+    // 6. Obtain function signature and rearrange expression according to the given signature.
+    val runtimeClass = scala.reflect.classTag[T].runtimeClass
+    val targetModuleSymbol = currentMirror.classSymbol(runtimeClass).companion
+    val parentClass = scala.reflect.classTag[SupportsNamedArguments].runtimeClass
+    val parentSymbol = currentMirror.classSymbol(parentClass)
+
+    targetModuleSymbol match {
+      case scala.reflect.runtime.universe.NoSymbol =>
+        throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
+      case _ =>
+        val moduleClassSymbol = targetModuleSymbol.asModule.moduleClass.asClass
+        if (moduleClassSymbol.baseClasses.contains(parentSymbol)) {
+          if (currentMirror.runtimeClass(moduleClassSymbol).getEnclosingClass != null) {
+            throw QueryCompilationErrors.cannotObtainCompanionObjectInstance(functionName)
+          }
+          val instance = currentMirror.reflectModule(targetModuleSymbol.asModule)
+            .instance.asInstanceOf[SupportsNamedArguments]
+          if (instance.functionSignatures.size != 1) {
+            throw QueryCompilationErrors.multipleFunctionSignatures(
+              functionName, instance.functionSignatures)
+          }
+          instance.rearrange(instance.functionSignatures.head, expressions, functionName)
+        } else {
+          throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
+        }
+    }
+  }
+
+  // Exposed for testing
+  final def defaultRearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters = functionSignature.parameters
+    val firstNamedArgIdx = args.indexWhere(_.isInstanceOf[NamedArgumentExpression])
+    val (positionalArgs, namedArgs) = args.splitAt(firstNamedArgIdx)
+    val namedParameters = parameters.drop(positionalArgs.size)
+
+    // Performing some checking to ensure valid argument list
+    val allParameterNames: Seq[String] = parameters.map(_.name)
+    val parameterNamesSet: Set[String] = allParameterNames.toSet
+    val assignedParameterSet = collection.mutable.Set[String](
+      allParameterNames.take(positionalArgs.size): _*)
+    for (arg <- namedArgs) {
+      arg match {
+        case namedArg: NamedArgumentExpression =>
+          if (assignedParameterSet.contains(namedArg.key)) {
+            throw QueryCompilationErrors.duplicateRoutineParameterAssignment(
+              functionName, namedArg.key)
+          }
+          if (!parameterNamesSet.contains(namedArg.key)) {
+            throw QueryCompilationErrors.unrecognizedParameterName(functionName, namedArg.key)
+          }
+          assignedParameterSet.add(namedArg.key)
+        case _ =>
+          throw QueryCompilationErrors.unexpectedPositionalArgument(functionName)
+      }
+    }
+
+    // Construct a map from argument name to value for argument rearrangement
+    val namedArgMap = namedArgs.map { arg =>
+      val namedArg = arg.asInstanceOf[NamedArgumentExpression]
+      namedArg.key -> namedArg.value
+    }.toMap
+
+    // Rearrange named arguments to match their positional order
+    val rearrangedNamedArgs = namedParameters.map { param =>
+      namedArgMap.getOrElse(
+        param.name,
+        if (param.default.isEmpty) {
+          throw QueryCompilationErrors.requiredParameterNotFound(functionName, param.name)
+        } else {

Review Comment:
   We can remove the else, but then for some reason, what is returned is ```Seq[Object]``` instead. It's easier to wrap it in an else as the compiler then won't complain.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] anchovYu commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "anchovYu (via GitHub)" <gi...@apache.org>.
anchovYu commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1253721448


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -1781,6 +1787,12 @@
       "Not allowed to implement multiple UDF interfaces, UDF class <className>."
     ]
   },
+  "NAMED_ARGUMENTS_NOT_SUPPORTED" : {
+    "message" : [
+      "Named arguments are not supported for function <functionName>; please retry the query with positional arguments to the function call instead or enable the feature using the SQL config \"spark.sql.allowNamedFunctionArguments\"."

Review Comment:
   I don't think this error is triggered when the config is false right? It is only because the function itself doesn't implement the named argument trait.



##########
sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql:
##########
@@ -1,5 +1,36 @@
+-- Test for named arguments for Mask
 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-@$#');
+
+-- Test for named arguments for CountMinSketchAgg
+create temporary view t2 as select * from values
+  ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'),
+  ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'),
+  ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null),
+  ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'),
+  ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'),
+  ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'),
+  ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'),
+  ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null)
+  as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i);
+
+SELECT hex(count_min_sketch(t2d, seed => 1, epsilon => 0.5d, confidence => 0.5d)) FROM t2;

Review Comment:
   Could you add a few more successful case like you did for `mask`?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * The class which companion objects of function expression may implement to
+ * support named arguments for that function expression.
+ *
+ * Example:
+ *  object CountMinSketchAgg extends SupportsNamedArguments {
+ *    final val functionSignature = FunctionSignature(Seq(
+ *      NamedArgument("column",
+ *          FixedArgumentType(TypeCollection(IntegralType, StringType, BinaryType))),
+ *      NamedArgument("epsilon", FixedArgumentType(DoubleType)),
+ *      NamedArgument("confidence", FixedArgumentType(DoubleType)),
+ *      NamedArgument("seed", FixedArgumentType(IntegerType))
+ *    ))
+ *    override def functionSignatures: Seq[FunctionSignature] = Seq(functionSignature)
+ *  }
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use this provided implementation as it is consistent with
+   * the SQL standard. If absolutely necessary the developer can choose to override the default
+   * behavior for additional flexibility.
+   *
+   * @param expectedSignature Function signature that denotes positional order of arguments
+   * @param providedArguments The sequence of expressions from function invocation
+   * @param functionName The name of the function invoked for debugging purposes
+   * @return positional order of arguments according to FunctionSignature obtained
+   *         by changing the order of the above provided arguments
+   */
+  protected def rearrange(
+      expectedSignature: FunctionSignature,
+      providedArguments: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(expectedSignature, providedArguments, functionName)
+  }
+}
+
+object SupportsNamedArguments {
+
+  /**
+   * Given a generic type, we check if the companion object of said type exists.
+   * If that object extends the trait [[SupportsNamedArguments]], then we rearrange
+   * the expressions in the order specified by the object.
+   *
+   * It is here we resubstitute [[Unevaluable]] [[NamedArgumentExpression]]s with
+   * normal expressions. This method will produce an positional argument list which
+   * is equivalent to the original argumnet list, except the expressions are now
+   * fit for consumption by [[ResolveFunctions]]
+   *
+   * @param expressions The list of positional and named argument expressions
+   * @tparam T The actual expression class.
+   * @return positional argument list
+   */
+  final def getRearrangedExpressions[T <: Expression : ClassTag](
+      expressions: Seq[Expression], functionName: String): Seq[Expression] = {
+
+    if (!expressions.exists(_.isInstanceOf[NamedArgumentExpression])) {
+      return expressions
+    }
+
+    import scala.reflect.runtime.currentMirror
+
+    // This code heavily utilizes Scala reflection which is unfamiliar to most developers.
+    // Here are the steps of this function:
+    // 1. Obtain the module symbol for the companion object of the function expression.
+    // 2. Obtain the module class symbol that represents the companion object.
+    // 3. Check if the base classes of the module class symbol contains SupportsNamedArguments.
+    //    This checks if the companion object is an implementor of SupportsNamedArguments.
+    // 4. Check if the module class symbol is a top level object. Reflection is unable to
+    //    obtain a companion object instance if it is member of some enclosing class unless
+    //    instance of said enclosing class is provided which we do not have.
+    // 5. Use reflection to obtain instance of companion object and perform immediate cast to
+    //    SupportsNamedArguments as it is already verified the cast is safe.
+    // 6. Obtain function signature and rearrange expression according to the given signature.
+    val runtimeClass = scala.reflect.classTag[T].runtimeClass
+    val targetModuleSymbol = currentMirror.classSymbol(runtimeClass).companion
+    val parentClass = scala.reflect.classTag[SupportsNamedArguments].runtimeClass
+    val parentSymbol = currentMirror.classSymbol(parentClass)
+
+    if(targetModuleSymbol == scala.reflect.runtime.universe.NoSymbol) {
+      throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
+    }
+
+    val moduleClassSymbol = targetModuleSymbol.asModule.moduleClass.asClass
+    if (!moduleClassSymbol.baseClasses.contains(parentSymbol)) {
+      throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
+    }
+    if (currentMirror.runtimeClass(moduleClassSymbol).getEnclosingClass != null) {
+      throw QueryCompilationErrors.cannotObtainCompanionObjectInstance(functionName)
+    }
+    val instance = currentMirror.reflectModule(targetModuleSymbol.asModule)
+      .instance.asInstanceOf[SupportsNamedArguments]
+    if (instance.functionSignatures.size != 1) {
+      throw QueryCompilationErrors.multipleFunctionSignatures(
+        functionName, instance.functionSignatures)
+    }
+    instance.rearrange(instance.functionSignatures.head, expressions, functionName)
+  }
+
+  final def defaultRearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters: Seq[NamedArgument] = functionSignature.parameters
+    val firstNamedArgIdx: Int = args.indexWhere(_.isInstanceOf[NamedArgumentExpression])
+    val (positionalArgs, namedArgs) = args.splitAt(firstNamedArgIdx)
+    val namedParameters: Seq[NamedArgument] = parameters.drop(positionalArgs.size)
+
+    // Performing some checking to ensure valid argument list
+    val allParameterNames: Seq[String] = parameters.map(_.name)
+    val parameterNamesSet: Set[String] = allParameterNames.toSet
+    val assignedParameterSet = collection.mutable.Set[String](
+      allParameterNames.take(positionalArgs.size): _*)
+    for (arg <- namedArgs) {
+      arg match {
+        case namedArg: NamedArgumentExpression =>
+          if (assignedParameterSet.contains(namedArg.key)) {
+            throw QueryCompilationErrors.duplicateRoutineParameterAssignment(
+              functionName, namedArg.key)
+          }
+          if (!parameterNamesSet.contains(namedArg.key)) {
+            throw QueryCompilationErrors.unrecognizedParameterName(functionName, namedArg.key,
+              parameterNamesSet.toSeq)
+          }
+          assignedParameterSet.add(namedArg.key)
+        case _ =>
+          throw QueryCompilationErrors.unexpectedPositionalArgument(functionName)
+      }
+    }
+
+    // Construct a map from argument name to value for argument rearrangement
+    val namedArgMap = namedArgs.map { arg =>
+      val namedArg = arg.asInstanceOf[NamedArgumentExpression]
+      namedArg.key -> namedArg.value
+    }.toMap
+
+    // Rearrange named arguments to match their positional order
+    val rearrangedNamedArgs: Seq[Expression] = namedParameters.map { param =>
+      namedArgMap.getOrElse(
+        param.name,
+        if (param.default.isEmpty) {
+          throw QueryCompilationErrors.requiredParameterNotFound(functionName, param.name)
+        } else {
+          param.default.get
+        }
+      )
+    }
+    positionalArgs ++ rearrangedNamedArgs
+  }
+}
+
+/**
+ * Identifies which forms of provided argument values are expected for each call
+ * to the associated SQL function
+ */
+trait NamedArgumentType
+
+/**
+ * Represents a named argument that expects a scalar value of one specific DataType
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * Represents a parameter of a function expression. Function expressions should use this class
+ * to construct the argument lists returned in [[SupportsNamedArguments.functionSignatures]]
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+                          name: String,

Review Comment:
   ```suggestion
   case class NamedArgument(
       name: String,
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala:
##########
@@ -320,4 +321,18 @@ object Mask {
       case _ => maskedChar(c, maskOther)
     }
   }
+  override def functionSignatures: Seq[FunctionSignature] = {
+    val strArg = NamedArgument("str", FixedArgumentType(StringType))
+    val upperCharArg = NamedArgument("upperChar",

Review Comment:
   ```suggestion
       val upperCharArg = NamedArgument(
         name = "upperChar",
         dataType = FixedArgumentType(StringType),
         ...
   ```



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2333,6 +2357,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. Here are some potential candidates: \n<arguments>"

Review Comment:
   ```suggestion
         "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>]."
   ```
   Let's make it aligned with other `WITH_SUGGESTION` error classes.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1254681711


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -674,6 +674,12 @@
     ],
     "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>."
+    ],
+    "sqlState" : "4274K"

Review Comment:
   Yep, consulted with him, and I have added the 4274K state documentation to the table.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1261777459


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -1015,6 +1050,62 @@ object TableFunctionRegistry {
   val functionSet: Set[FunctionIdentifier] = builtin.listFunction().toSet
 }
 
-trait ExpressionBuilder {
-  def build(funcName: String, expressions: Seq[Expression]): Expression
+trait Builder[T] {
+  /**
+   * A method that returns the signatures of overloads that is associated with this function
+   *
+   * @return a list of function signatures
+   */
+  def functionSignatures: Option[Seq[FunctionSignature]] = None
+
+  /**
+   * This function rearranges the arguments provided during function invocation in positional order
+   * according to the function signature. This method will fill in the default values if optional
+   * parmaeters do not have their values specified. Any function which supports named arguments
+   * will have this routine invoked, even if no named arguments are present in the argument list.
+   * This is done to eliminate constructor overloads in some methods which use them for default
+   * values prior to the implementation of the named argument framework. This function will also
+   * check if the number of arguments are correct. If that is not the case, then an error will be thrown.
+   *
+   * IMPORTANT: This method will be called before the [[Builder.build]] method is invoked. It is
+   * guaranteed that the expressions provided to the [[Builder.build]] functions forms a valid set
+   * of argument expressions that can be used in the construction of the function expression.
+   *
+   * @param expectedSignature The method signature which we rearrange our arguments according to
+   * @param providedArguments The list of arguments passed from function invocation
+   * @param functionName The name of the function
+   * @return The rearranged arugument list with arguments in positional order
+   */
+  def rearrange(
+      expectedSignature: FunctionSignature,
+      providedArguments: Seq[Expression],
+      functionName: String) : Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(expectedSignature, providedArguments, functionName)
+  }
+
+  def build(funcName: String, expressions: Seq[Expression]): T
+}
+
+/**
+ * A trait used for scalar valued functions that defines how their expression representations
+ * are constructed in [[FunctionRegistry]]
+ */
+trait ExpressionBuilder extends Builder[Expression]
+
+/**
+ * A trait used for table valued functions that defines how their expression representations
+ * are constructed in [[FunctionRegistry]]
+ */
+trait GeneratorBuilder extends Builder[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

Review Comment:
   I added this because the build function in ExpressionBuilder requires it. It's to be consistent with the previous ```ExpressionBuilder#build(name, arguments)``` method.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] anchovYu commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "anchovYu (via GitHub)" <gi...@apache.org>.
anchovYu commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1258881422


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -897,7 +895,20 @@ 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 =
+        if (builder.functionSignatures != Nil) {

Review Comment:
   After making it a `Option` following Daniel's comment, we will use `isDefined` here



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -979,9 +990,7 @@ object TableFunctionRegistry {
       : (String, (ExpressionInfo, TableFunctionBuilder)) = {
     val (info, builder) = FunctionRegistryBase.build[T](name, since = None)
     val newBuilder = (expressions: Seq[Expression]) => {
-      val rearrangedExpressions =
-        SupportsNamedArguments.getRearrangedExpressions[T](expressions, name)
-      val generator = builder(rearrangedExpressions)
+      val generator = builder(expressions)

Review Comment:
   What's the plan to support these for named arguments?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -71,73 +69,17 @@ abstract class SupportsNamedArguments {
 }
 
 object SupportsNamedArguments {
-
-  /**
-   * Given a generic type, we check if the companion object of said type exists.
-   * If that object extends the trait [[SupportsNamedArguments]], then we rearrange
-   * the expressions in the order specified by the object.
-   *
-   * It is here we resubstitute [[Unevaluable]] [[NamedArgumentExpression]]s with
-   * normal expressions. This method will produce an positional argument list which
-   * is equivalent to the original argumnet list, except the expressions are now
-   * fit for consumption by [[ResolveFunctions]]
-   *
-   * @param expressions The list of positional and named argument expressions
-   * @tparam T The actual expression class.
-   * @return positional argument list
-   */
-  final def getRearrangedExpressions[T : ClassTag](
-      expressions: Seq[Expression], functionName: String): Seq[Expression] = {
-
-    if (!expressions.exists(_.isInstanceOf[NamedArgumentExpression])) {
-      return expressions
-    }
-
-    import scala.reflect.runtime.currentMirror
-
-    // This code heavily utilizes Scala reflection which is unfamiliar to most developers.
-    // Here are the steps of this function:
-    // 1. Obtain the module symbol for the companion object of the function expression.
-    // 2. Obtain the module class symbol that represents the companion object.
-    // 3. Check if the base classes of the module class symbol contains SupportsNamedArguments.
-    //    This checks if the companion object is an implementor of SupportsNamedArguments.
-    // 4. Check if the module class symbol is a top level object. Reflection is unable to
-    //    obtain a companion object instance if it is member of some enclosing class unless
-    //    instance of said enclosing class is provided which we do not have.
-    // 5. Use reflection to obtain instance of companion object and perform immediate cast to
-    //    SupportsNamedArguments as it is already verified the cast is safe.
-    // 6. Obtain function signature and rearrange expression according to the given signature.
-    val runtimeClass = scala.reflect.classTag[T].runtimeClass
-    val targetModuleSymbol = currentMirror.classSymbol(runtimeClass).companion
-    val parentClass = scala.reflect.classTag[SupportsNamedArguments].runtimeClass
-    val parentSymbol = currentMirror.classSymbol(parentClass)
-
-    if(targetModuleSymbol == scala.reflect.runtime.universe.NoSymbol) {
-      throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
-    }
-
-    val moduleClassSymbol = targetModuleSymbol.asModule.moduleClass.asClass
-    if (!moduleClassSymbol.baseClasses.contains(parentSymbol)) {
-      throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
-    }
-    if (currentMirror.runtimeClass(moduleClassSymbol).getEnclosingClass != null) {
-      throw QueryCompilationErrors.cannotObtainCompanionObjectInstance(functionName)
-    }
-    val instance = currentMirror.reflectModule(targetModuleSymbol.asModule)
-      .instance.asInstanceOf[SupportsNamedArguments]
-    if (instance.functionSignatures.size != 1) {
-      throw QueryCompilationErrors.multipleFunctionSignatures(
-        functionName, instance.functionSignatures)
-    }
-    instance.rearrange(instance.functionSignatures.head, expressions, functionName)
-  }
-
   final def defaultRearrange(functionSignature: FunctionSignature,
       args: Seq[Expression],
       functionName: String): Seq[Expression] = {
     val parameters: Seq[NamedArgument] = functionSignature.parameters
     val firstNamedArgIdx: Int = args.indexWhere(_.isInstanceOf[NamedArgumentExpression])
-    val (positionalArgs, namedArgs) = args.splitAt(firstNamedArgIdx)
+    val (positionalArgs, namedArgs) =
+      if (firstNamedArgIdx == -1) {

Review Comment:
   If it can't find first named argument, does it just return the original arg list without reordering? If so you can return early.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala:
##########
@@ -344,4 +346,13 @@ object Mask extends SupportsNamedArguments {
       strArg, upperCharArg, lowerCharArg, digitCharArg, otherCharArg))
     Seq(functionSignature)
   }
+
+  override def build(funcName: String, expressions: Seq[Expression]): Expression = {
+    if (expressions.length < 1 || expressions.length > 5) {
+      throw QueryCompilationErrors.wrongNumArgsError(
+        funcName, Seq(1, 2, 3, 4, 5), expressions.length)
+    }
+    new Mask(expressions(0), expressions(1), expressions(2), expressions(3), expressions(4))

Review Comment:
   The expression builder way shrifts the responsibility to find a 'right' constructor (constructor with the right number of arguments) from the `FunctionRegistryBase.build` that uses reflection, to the expression builder itself. I'm not sure if this is a preferred way @cloud-fan  - to really make this `mask` function work, we will need to write argument number matching in the `build` function of the expression builder, e.g. `if (expressions.size == 2) .. `.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala:
##########
@@ -344,4 +346,13 @@ object Mask extends SupportsNamedArguments {
       strArg, upperCharArg, lowerCharArg, digitCharArg, otherCharArg))
     Seq(functionSignature)
   }
+
+  override def build(funcName: String, expressions: Seq[Expression]): Expression = {
+    if (expressions.length < 1 || expressions.length > 5) {
+      throw QueryCompilationErrors.wrongNumArgsError(
+        funcName, Seq(1, 2, 3, 4, 5), expressions.length)
+    }
+    new Mask(expressions(0), expressions(1), expressions(2), expressions(3), expressions(4))

Review Comment:
   What happen when there is no named argument but users provide only 3 arguments?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala:
##########
@@ -344,4 +346,13 @@ object Mask extends SupportsNamedArguments {
       strArg, upperCharArg, lowerCharArg, digitCharArg, otherCharArg))
     Seq(functionSignature)
   }
+
+  override def build(funcName: String, expressions: Seq[Expression]): Expression = {
+    if (expressions.length < 1 || expressions.length > 5) {
+      throw QueryCompilationErrors.wrongNumArgsError(
+        funcName, Seq(1, 2, 3, 4, 5), expressions.length)
+    }
+    new Mask(expressions(0), expressions(1), expressions(2), expressions(3), expressions(4))

Review Comment:
   We can't affect the current usage without named arguments. e.g. we can't reject these cases. It will become a breaking change.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1262839893


##########
sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out:
##########
@@ -327,13 +327,14 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
-  "sqlState" : "42605",
+  "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+  "sqlState" : "42K09",
   "messageParameters" : {
-    "actualNum" : "2",
-    "docroot" : "https://spark.apache.org/docs/latest",
-    "expectedNum" : "1",
-    "functionName" : "`explode`"
+    "inputSql" : "\"1\"",
+    "inputType" : "\"INT\"",
+    "paramIndex" : "1",
+    "requiredType" : "(\"ARRAY\" or \"MAP\")",
+    "sqlExpr" : "\"explode(1)\""
   },
   "queryContext" : [ {
     "objectType" : "",

Review Comment:
   Sure, that's fine. I fixed the tests now so the errors remain consistent.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1263152648


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/FunctionBuilderBase.scala:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+/**
+ * This is a base trait that is used for implementing builder classes that can be used to construct
+ * expressions or logical plans depending on if it is a table-valued or scala-valued function.

Review Comment:
   ```suggestion
    * expressions or logical plans depending on if it is a table-valued or scalar function.
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1261772618


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+object SupportsNamedArguments {
+  final def defaultRearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters: Seq[NamedArgument] = functionSignature.parameters
+    val firstNamedArgIdx: Int = args.indexWhere(_.isInstanceOf[NamedArgumentExpression])
+    val (positionalArgs, namedArgs) =
+      if (firstNamedArgIdx == -1) {
+        (args, Nil)

Review Comment:
   This is actually intentional since this fits very nicely with the rest of the named argument checking that is done later in the method. It's perfectly safe for Nil to be passed as an argument since that represents an empty Seq and this typically happens in situations where functions with only positional arguments provided are invoked. I don't think the option is necessary.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1262249027


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+object SupportsNamedArguments {
+  /**
+   * This method is the default routine which rearranges the arguments in positional order according
+   * to the function signature provided. This will also fill in any default values that exists for
+   * optional arguments. This method will also be invoked even if there are no named arguments in
+   * the argument list.
+   *
+   * @param functionSignature The function signature that defines the positional ordering
+   * @param args The argument list provided in function invocation
+   * @param functionName The name of the function
+   * @return A list of arguments rearranged in positional order defined by the provided signature
+   */
+  final def defaultRearrange(
+      functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters: Seq[NamedArgument] = functionSignature.parameters
+    val firstOptionalParamIndex: Int = parameters.indexWhere(_.default.isDefined)
+    if (firstOptionalParamIndex != -1 &&
+        parameters.drop(firstOptionalParamIndex).exists(_.default.isEmpty)) {
+      throw QueryCompilationErrors.unexpectedRequiredParameterInFunctionSignature(
+        functionName, functionSignature)
+    }
+
+    val firstNamedArgIdx: Int = args.indexWhere(_.isInstanceOf[NamedArgumentExpression])
+    val (positionalArgs, namedArgs) =
+      if (firstNamedArgIdx == -1) {
+        (args, Nil)
+      } else {
+        args.splitAt(firstNamedArgIdx)
+      }
+    val namedParameters: Seq[NamedArgument] = parameters.drop(positionalArgs.size)
+
+    // The following loop checks for the following:
+    // 1. Unrecognized parameter names
+    // 2. Duplicate routine parameter assignments
+    val allParameterNames: Seq[String] = parameters.map(_.name)

Review Comment:
   to confirm: the parameter name is case-sensitive. Is it also true in other systems or the SQL standard? cc @srielau 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1262243416


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+object SupportsNamedArguments {
+  /**
+   * This method is the default routine which rearranges the arguments in positional order according
+   * to the function signature provided. This will also fill in any default values that exists for
+   * optional arguments. This method will also be invoked even if there are no named arguments in
+   * the argument list.
+   *
+   * @param functionSignature The function signature that defines the positional ordering
+   * @param args The argument list provided in function invocation
+   * @param functionName The name of the function
+   * @return A list of arguments rearranged in positional order defined by the provided signature
+   */
+  final def defaultRearrange(
+      functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters: Seq[NamedArgument] = functionSignature.parameters
+    val firstOptionalParamIndex: Int = parameters.indexWhere(_.default.isDefined)
+    if (firstOptionalParamIndex != -1 &&
+        parameters.drop(firstOptionalParamIndex).exists(_.default.isEmpty)) {

Review Comment:
   nit: `parameters.dropWhile(_.default.isEmpty).exists(_.default.isEmpty)` is simpler



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1262251756


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+object SupportsNamedArguments {

Review Comment:
   Do we have this object for ease of test? It looks to me that we can move the new `trait Builder[T]` to a new file and define the `defaultRearrange` function there.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1261740266


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -887,6 +887,26 @@ object FunctionRegistry {
     (name, (expressionInfo, newBuilder))
   }
 
+  private[FunctionRegistry$] def rearrangeExpressions[T <: Builder[_]](

Review Comment:
   please add a comment for this method?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+object SupportsNamedArguments {
+  final def defaultRearrange(functionSignature: FunctionSignature,

Review Comment:
   please add a descriptive comment for this method? Also, move `functionSignature: FunctionSignature` to the next line with an indent of  +4 spaces per style recommendations?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+object SupportsNamedArguments {
+  final def defaultRearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters: Seq[NamedArgument] = functionSignature.parameters
+    val firstNamedArgIdx: Int = args.indexWhere(_.isInstanceOf[NamedArgumentExpression])
+    val (positionalArgs, namedArgs) =
+      if (firstNamedArgIdx == -1) {
+        (args, Nil)

Review Comment:
   It's safer to use `None` and `Option` types instead of `null` or `Nil` in Scala, since the API prompts invokers to check for the object's presence before extracting.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala:
##########
@@ -344,4 +346,13 @@ object Mask extends SupportsNamedArguments {
       strArg, upperCharArg, lowerCharArg, digitCharArg, otherCharArg))
     Seq(functionSignature)
   }
+
+  override def build(funcName: String, expressions: Seq[Expression]): Expression = {
+    if (expressions.length < 1 || expressions.length > 5) {
+      throw QueryCompilationErrors.wrongNumArgsError(
+        funcName, Seq(1, 2, 3, 4, 5), expressions.length)
+    }
+    new Mask(expressions(0), expressions(1), expressions(2), expressions(3), expressions(4))

Review Comment:
   Yes, +1 on this approach as well.
   
   Eventually, we can considering putting more required properties in the function signatures, such that most function expression classes do not have to implement any matching logic between provided vs. required arguments at all, and it will be handled generically by the analyzer instead. For now, keeping just the argument names and optional default values in the function signature lets us accomplish the goal of supporting named arguments without breaking any existing use cases.



##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/NamedArgumentFunctionSuite.scala:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.analysis
+
+// import org.apache.spark.SparkThrowable

Review Comment:
   please uncomment or delete commented-out code



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+object SupportsNamedArguments {
+  final def defaultRearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters: Seq[NamedArgument] = functionSignature.parameters
+    val firstNamedArgIdx: Int = args.indexWhere(_.isInstanceOf[NamedArgumentExpression])
+    val (positionalArgs, namedArgs) =
+      if (firstNamedArgIdx == -1) {
+        (args, Nil)
+      } else {
+        args.splitAt(firstNamedArgIdx)
+      }
+    val namedParameters: Seq[NamedArgument] = parameters.drop(positionalArgs.size)
+
+    // Performing some checking to ensure valid argument list

Review Comment:
   nit: please make each comment a complete sentence and end it with a period



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+object SupportsNamedArguments {
+  final def defaultRearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters: Seq[NamedArgument] = functionSignature.parameters
+    val firstNamedArgIdx: Int = args.indexWhere(_.isInstanceOf[NamedArgumentExpression])
+    val (positionalArgs, namedArgs) =
+      if (firstNamedArgIdx == -1) {
+        (args, Nil)
+      } else {
+        args.splitAt(firstNamedArgIdx)
+      }
+    val namedParameters: Seq[NamedArgument] = parameters.drop(positionalArgs.size)
+
+    // Performing some checking to ensure valid argument list
+    val allParameterNames: Seq[String] = parameters.map(_.name)
+    val parameterNamesSet: Set[String] = allParameterNames.toSet
+    val positionalParametersSet = allParameterNames.take(positionalArgs.size).toSet
+    val namedParametersSet = collection.mutable.Set[String]()
+
+    for (arg <- namedArgs) {
+      arg match {
+        case namedArg: NamedArgumentExpression =>
+          val parameterName = namedArg.key
+          if (!parameterNamesSet.contains(parameterName)) {
+            throw QueryCompilationErrors.unrecognizedParameterName(functionName, namedArg.key,
+              parameterNamesSet.toSeq)
+          }
+          if (positionalParametersSet.contains(parameterName)) {
+            throw QueryCompilationErrors.positionalAndNamedArgumentDoubleReference(
+              functionName, namedArg.key)
+          }
+          if (namedParametersSet.contains(parameterName)) {
+            throw QueryCompilationErrors.doubleNamedArgumentReference(
+              functionName, namedArg.key)
+          }
+          namedParametersSet.add(namedArg.key)
+        case _ =>
+          throw QueryCompilationErrors.unexpectedPositionalArgument(functionName)
+      }
+    }
+
+    // Construct a map from argument name to value for argument rearrangement
+    val namedArgMap = namedArgs.map { arg =>
+      val namedArg = arg.asInstanceOf[NamedArgumentExpression]
+      namedArg.key -> namedArg.value
+    }.toMap
+
+    // Rearrange named arguments to match their positional order
+    val rearrangedNamedArgs: Seq[Expression] = namedParameters.map { param =>
+      namedArgMap.getOrElse(
+        param.name,
+        if (param.default.isEmpty) {
+          throw QueryCompilationErrors.requiredParameterNotFound(functionName, param.name)
+        } else {
+          param.default.get
+        }
+      )
+    }
+    positionalArgs ++ rearrangedNamedArgs
+  }
+}
+
+/**
+ * Identifies which forms of provided argument values are expected for each call
+ * to the associated SQL function
+ */
+trait NamedArgumentType
+
+/**
+ * Represents a named argument that expects a scalar value of one specific DataType
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType

Review Comment:
   this class is no longer referenced in this PR, since we removed the data types from the function signatures. Maybe just delete it for now, until we decide to add them back in a later PR?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -1015,6 +1050,62 @@ object TableFunctionRegistry {
   val functionSet: Set[FunctionIdentifier] = builtin.listFunction().toSet
 }
 
-trait ExpressionBuilder {
-  def build(funcName: String, expressions: Seq[Expression]): Expression
+trait Builder[T] {
+  /**
+   * A method that returns the signatures of overloads that is associated with this function

Review Comment:
   ```suggestion
      * A method that returns the signatures of overloads that are associated with this function.
      * Each function signature includes a list of parameters to which the analyzer can
      * compare a function call with provided arguments to determine if that function
      * call is a match for the function signature.
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+object SupportsNamedArguments {
+  final def defaultRearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters: Seq[NamedArgument] = functionSignature.parameters
+    val firstNamedArgIdx: Int = args.indexWhere(_.isInstanceOf[NamedArgumentExpression])
+    val (positionalArgs, namedArgs) =

Review Comment:
   ```suggestion
       val (positionalArgs: Seq[Expression], namedArgs: Option[Seq[Expression]]) =
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1263437569


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2445,6 +2481,12 @@
     ],
     "sqlState" : "428C4"
   },
+  "UNRECOGNIZED_PARAMETER_NAME" : {

Review Comment:
   ```suggestion
     "UNRECOGNIZED_ARGUMENT_NAME" : {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1253702408


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+    dataType: NamedArgumentType,
+    default: Option[Expression] = None)
+
+/**
+ * Represents a method signature and the list of arguments it receives as input.
+ * Currently, overloads are not supported and only one FunctionSignature is allowed
+ * per function expression.
+ *
+ * @param parameters The list of arguments which the function takes
+ */
+case class FunctionSignature(parameters: Seq[NamedArgument])
+
+/**
+ * The class which companion objects of function expression implement to
+ * support named arguments for that function expression.
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use the provided version rearrange as it is consistent with
+   * the SQL standard. If absolutely necessary the developer can choose to override the default
+   * behavior for additional flexibility.
+   *
+   * @param functionSignature Function signature that denotes positional order of arguments
+   * @param args The sequence of expressions from function invocation
+   * @param functionName The name of the function invoked for debugging purposes
+   * @return positional order of arguments according to FunctionSignature
+   */
+  protected def rearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(functionSignature, args, functionName)
+  }
+}
+
+object SupportsNamedArguments {
+
+  /**
+   * Given a generic type, we check if the companion object of said type exists.
+   * If that object extends the trait [[SupportsNamedArguments]], then we rearrange
+   * the expressions in the order specified by the object.
+   *
+   * It is here we resubstitute [[Unevaluable]] [[NamedArgumentExpression]]s with
+   * normal expressions. This method will produce an positional argument list which
+   * is equivalent to the original argumnet list, except the expressions are now
+   * fit for consumption by [[ResolveFunctions]]
+   *
+   * @param expressions The list of positional and named argument expressions
+   * @tparam T The actual expression class.
+   * @return positional argument list
+   */
+  final def getRearrangedExpressions[T <: Expression : ClassTag](
+      expressions: Seq[Expression], functionName: String): Seq[Expression] = {
+
+    if (!expressions.exists(_.isInstanceOf[NamedArgumentExpression])) {
+      return expressions
+    }
+
+    import scala.reflect.runtime.currentMirror
+
+    // This code heavily utilizes Scala reflection which is unfamiliar to most developers.
+    // Here are the steps of this function:
+    // 1. Obtain the module symbol for the companion object of the function expression.
+    // 2. Obtain the module class symbol that represents the companion object.
+    // 3. Check if the base classes of the module class symbol contains SupportsNamedArguments.
+    //    This checks if the companion object is an implementor of SupportsNamedArguments.
+    // 4. Check if the module class symbol is a top level object. Reflection is unable to
+    //    obtain a companion object instance if it is member of some enclosing class unless
+    //    instance of said enclosing class is provided which we do not have.
+    // 5. Use reflection to obtain instance of companion object and perform immediate cast to
+    //    SupportsNamedArguments as it is already verified the cast is safe.
+    // 6. Obtain function signature and rearrange expression according to the given signature.
+    val runtimeClass = scala.reflect.classTag[T].runtimeClass
+    val targetModuleSymbol = currentMirror.classSymbol(runtimeClass).companion
+    val parentClass = scala.reflect.classTag[SupportsNamedArguments].runtimeClass
+    val parentSymbol = currentMirror.classSymbol(parentClass)
+
+    targetModuleSymbol match {
+      case scala.reflect.runtime.universe.NoSymbol =>

Review Comment:
   True. I can do that.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1254770625


##########
sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql:
##########
@@ -1,5 +1,36 @@
+-- Test for named arguments for Mask
 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-@$#');
+
+-- Test for named arguments for CountMinSketchAgg
+create temporary view t2 as select * from values
+  ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'),
+  ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'),
+  ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null),
+  ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'),
+  ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'),
+  ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'),
+  ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'),
+  ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null)
+  as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i);
+
+SELECT hex(count_min_sketch(t2d, seed => 1, epsilon => 0.5d, confidence => 0.5d)) FROM t2;
+
+-- Unexpected positional argument
 SELECT mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd');
+-- Duplicate parameter assignment

Review Comment:
   Added the test. Not too sure if this will do.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] anchovYu commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "anchovYu (via GitHub)" <gi...@apache.org>.
anchovYu commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1259998570


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala:
##########
@@ -208,3 +209,20 @@ case class CountMinSketchAgg(
       confidenceExpression = third,
       seedExpression = fourth)
 }
+
+object CountMinSketchAggExpressionBuilder extends ExpressionBuilder {
+  final val functionSignature = FunctionSignature(Seq(
+    NamedArgument("column",
+                  FixedArgumentType(TypeCollection(IntegralType, StringType, BinaryType))),
+    NamedArgument("epsilon", FixedArgumentType(DoubleType)),
+    NamedArgument("confidence", FixedArgumentType(DoubleType)),
+    NamedArgument("seed", FixedArgumentType(IntegerType))

Review Comment:
   I don't think the new framework will check the data types or the input requirements - this will still be the responsibility and functionality of original `checkInputDataTypes`. The new framework basically just reorder the arguments based on the **names**. In the current implementation it doesn't really use the type information in the signatures.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1258767723


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -1016,5 +1029,14 @@ object TableFunctionRegistry {
 }
 
 trait ExpressionBuilder {
+  def functionSignatures: Seq[FunctionSignature] = Nil
+
+  def rearrange(

Review Comment:
   Can you please add comments for each of these methods describing what they are for and who calls them?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala:
##########
@@ -321,3 +322,37 @@ object Mask {
     }
   }
 }
+
+object MaskExpressionBuilder extends ExpressionBuilder {
+  override def functionSignatures: Seq[FunctionSignature] = {
+    val strArg = NamedArgument("str", FixedArgumentType(StringType))
+    val upperCharArg = NamedArgument(
+      "upperChar",
+      FixedArgumentType(StringType),
+      Some(Literal(Mask.MASKED_UPPERCASE)))
+    val lowerCharArg = NamedArgument(
+      "lowerChar",
+      FixedArgumentType(StringType),
+      Some(Literal(Mask.MASKED_LOWERCASE)))
+    val digitCharArg = NamedArgument(
+      "digitChar",
+      FixedArgumentType(StringType),
+      Some(Literal(Mask.MASKED_DIGIT)))
+    val otherCharArg = NamedArgument(
+      "otherChar",
+      FixedArgumentType(StringType),
+      Some(Literal(Mask.MASKED_IGNORE)))
+    val functionSignature: FunctionSignature = FunctionSignature(Seq(
+      strArg, upperCharArg, lowerCharArg, digitCharArg, otherCharArg))
+    Seq(functionSignature)
+  }
+
+  override def build(funcName: String, expressions: Seq[Expression]): Expression = {
+    if (expressions.length < 1 || expressions.length > 5) {
+      throw QueryCompilationErrors.wrongNumArgsError(
+        funcName, Seq(1, 2, 3, 4, 5), expressions.length)
+    }
+    new Mask(expressions(0), expressions(1), expressions(2), expressions(3), expressions(4))

Review Comment:
   here it seems like by this point, since we provided an override for `functionSignatures`, that we have the guarantee that the `expressions` that show up here are all 5 expressions from the function signature, even if one or more of the provided arguments are missing (and we substitute the corresponding default expressions instead). Can you comment this on the `build` method inside the base `ExpressionBuilder` to make sure the semantics are clear?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -1016,5 +1029,14 @@ object TableFunctionRegistry {
 }
 
 trait ExpressionBuilder {
+  def functionSignatures: Seq[FunctionSignature] = Nil

Review Comment:
   using null/Nil is generally not advisable because if the user forgets to check, it leads to exceptions. Maybe make this return `Option[Seq[FunctionSignature]]` instead with `None` as the default instead?



##########
sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql:
##########
@@ -1,5 +1,36 @@
+-- Test for named arguments for Mask
 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-@$#');
+
+-- Test for named arguments for CountMinSketchAgg
+create temporary view t2 as select * from values
+  ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'),
+  ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'),
+  ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null),
+  ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'),
+  ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'),
+  ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'),
+  ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'),
+  ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null)
+  as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i);
+
+SELECT hex(count_min_sketch(t2d, seed => 1, epsilon => 0.5d, confidence => 0.5d)) FROM t2;
+
+-- Unexpected positional argument
 SELECT mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd');
+-- Duplicate parameter assignment

Review Comment:
   Where is the testing for this?



##########
sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql:
##########
@@ -1,5 +1,36 @@
+-- Test for named arguments for Mask
 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-@$#');
+
+-- Test for named arguments for CountMinSketchAgg
+create temporary view t2 as select * from values
+  ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'),
+  ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'),
+  ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null),
+  ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'),
+  ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'),
+  ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'),
+  ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'),
+  ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null)
+  as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i);
+
+SELECT hex(count_min_sketch(t2d, seed => 1, epsilon => 0.5d, confidence => 0.5d)) FROM t2;
+
+-- Unexpected positional argument

Review Comment:
   Ping on this comment :)



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * The class which companion objects of function expression may implement to
+ * support named arguments for that function expression. Please note that variadic final
+ * arguments are NOT supported for named arguments. Do not use for functions that
+ * has variadic final arguments!

Review Comment:
   ```suggestion
    * have variadic final arguments!
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1256520869


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala:
##########
@@ -431,6 +432,11 @@ case class Explode(child: Expression) extends ExplodeBase {
     copy(child = newChild)
 }
 
+object Explode extends SupportsNamedArguments {

Review Comment:
   Right now, we don't really have support for named arguments for final variadic argument. We can actually include support for it if needed. I think it just probably requires a bit of extra logic to handle the case. @dtenedor What do you think?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1256251978


##########
sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql:
##########
@@ -1,5 +1,36 @@
+-- Test for named arguments for Mask
 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-@$#');
+
+-- Test for named arguments for CountMinSketchAgg
+create temporary view t2 as select * from values
+  ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'),
+  ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'),
+  ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null),
+  ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'),
+  ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'),
+  ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'),
+  ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'),
+  ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'),
+  ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'),
+  ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null)
+  as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i);
+
+SELECT hex(count_min_sketch(t2d, seed => 1, epsilon => 0.5d, confidence => 0.5d)) FROM t2;
+
+-- Unexpected positional argument

Review Comment:
   We should probably make a separate error class for that to help allay confusion, the error message can explicitly mention that the argument was provided once by name and also one or more times positionally, and this is not allowed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonwang-db commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "allisonwang-db (via GitHub)" <gi...@apache.org>.
allisonwang-db commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1256488444


##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/NamedArgumentFunctionSuite.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.analysis
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.SparkThrowable
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, NamedArgumentExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
+import org.apache.spark.sql.catalyst.plans.logical.{FixedArgumentType, FunctionSignature, NamedArgument, SupportsNamedArguments}
+import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLId
+import org.apache.spark.sql.types.{DataType, StringType}
+
+
+case class DummyExpression() extends Expression {
+  override def nullable: Boolean = false
+  override def eval(input: InternalRow): Any = None
+  override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = null
+  override def dataType: DataType = null
+  override def children: Seq[Expression] = Nil
+  override protected def withNewChildrenInternal(
+    newChildren: IndexedSeq[Expression]): Expression = null
+}
+
+object DummyExpression extends SupportsNamedArguments {
+  def defaultFunctionSignature: FunctionSignature = {
+    FunctionSignature(Seq(NamedArgument("k1", FixedArgumentType(StringType)),
+      NamedArgument("k2", FixedArgumentType(StringType)),
+      NamedArgument("k3", FixedArgumentType(StringType)),
+      NamedArgument("k4", FixedArgumentType(StringType))))
+  }
+  override def functionSignatures: Seq[FunctionSignature] = {
+    Seq(defaultFunctionSignature)
+  }
+}
+
+case class SignaturesExpression() extends Expression {
+  override def nullable: Boolean = false
+  override def eval(input: InternalRow): Any = None
+  override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = null
+  override def dataType: DataType = null
+  override def children: Seq[Expression] = Nil
+  override protected def withNewChildrenInternal(
+    newChildren: IndexedSeq[Expression]): Expression = null
+}
+
+object SignaturesExpression extends SupportsNamedArguments {
+  override def functionSignatures: Seq[FunctionSignature] = Seq(null, null)
+}
+
+case class NoNamedArgumentsExpression() extends Expression {
+  override def nullable: Boolean = false
+  override def eval(input: InternalRow): Any = None
+  override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = null
+  override def dataType: DataType = null
+  override def children: Seq[Expression] = Nil
+  override protected def withNewChildrenInternal(
+    newChildren: IndexedSeq[Expression]): Expression = null
+}
+
+class NamedArgumentFunctionSuite extends AnalysisTest {
+
+  final val k1Arg = Literal("v1")
+  final val k2Arg = NamedArgumentExpression("k2", Literal("v2"))
+  final val k3Arg = NamedArgumentExpression("k3", Literal("v3"))
+  final val k4Arg = NamedArgumentExpression("k4", Literal("v4"))
+  final val args = Seq(k1Arg, k4Arg, k2Arg, k3Arg)
+  final val expectedSeq = Seq(Literal("v1"), Literal("v2"), Literal("v3"), Literal("v4"))
+
+  def rearrangeExpressions[T <: Expression : ClassTag](
+        expressions: Seq[Expression], functionName: String = "function"): Seq[Expression] = {
+    SupportsNamedArguments.getRearrangedExpressions[T](expressions, functionName)
+  }
+
+  test("Check rearrangement of expressions") {
+    val rearrangedArgs = SupportsNamedArguments.defaultRearrange(
+      DummyExpression.defaultFunctionSignature, args, "function")
+    for ((returnedArg, expectedArg) <- rearrangedArgs.zip(expectedSeq)) {
+      assert(returnedArg == expectedArg)
+    }
+  }
+
+  test("Check inheritance restrictions are enforced.") {
+    val rearrangedArgs = rearrangeExpressions[DummyExpression](args)
+    for ((returnedArg, expectedArg) <- rearrangedArgs.zip(expectedSeq)) {
+      assert(returnedArg == expectedArg)
+    }
+  }
+
+  private def parseRearrangeException(functionSignature: FunctionSignature,
+                                      expressions: Seq[Expression],
+                                      functionName: String = "function"): SparkThrowable = {

Review Comment:
   nit: space



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala:
##########
@@ -431,6 +432,11 @@ case class Explode(child: Expression) extends ExplodeBase {
     copy(child = newChild)
 }
 
+object Explode extends SupportsNamedArguments {

Review Comment:
   Also, for a table function like `stack(n, expr1, ..., exprk)`, how would you add named arguments for it?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala:
##########
@@ -431,6 +432,11 @@ case class Explode(child: Expression) extends ExplodeBase {
     copy(child = newChild)
 }
 
+object Explode extends SupportsNamedArguments {

Review Comment:
   Why should the 'explode' function support named arguments if it only has one argument?
   ```
   explode(expr)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] anchovYu commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "anchovYu (via GitHub)" <gi...@apache.org>.
anchovYu commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1253421717


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -674,6 +674,12 @@
     ],
     "sqlState" : "23505"
   },
+  "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT" : {
+    "message" : [
+      "Call to function <functionName> is invalid because it includes multiple argument assignments to the same name <parameterName>."
+    ],
+    "sqlState" : "42704K"

Review Comment:
   I feel that some of my previous comments are naturally collapsed so you can't see them without expanding. I'll copy those to this new PR.
   
   The error code should be 5 characters according to https://github.com/apache/spark/tree/master/common/utils/src/main/resources/error. 



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala:
##########
@@ -320,4 +321,18 @@ object Mask {
       case _ => maskedChar(c, maskOther)
     }
   }
+  override def functionSignatures: Seq[FunctionSignature] = {
+    val strArg = NamedArgument("str", FixedArgumentType(StringType))
+    val upperCharArg = NamedArgument("upperChar",

Review Comment:
   nit: For code style could you split each argument on a new line?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(name: String,
+    dataType: NamedArgumentType,
+    default: Option[Expression] = None)
+
+/**
+ * Represents a method signature and the list of arguments it receives as input.
+ * Currently, overloads are not supported and only one FunctionSignature is allowed
+ * per function expression.
+ *
+ * @param parameters The list of arguments which the function takes
+ */
+case class FunctionSignature(parameters: Seq[NamedArgument])
+
+/**
+ * The class which companion objects of function expression implement to
+ * support named arguments for that function expression.
+ */
+abstract class SupportsNamedArguments {
+  def functionSignatures: Seq[FunctionSignature]

Review Comment:
   Could you add a comment here noting that now by default only one signature is allowed, with some reasoning?



##########
sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql:
##########
@@ -1,5 +1,13 @@
+-- Test for named arguments
 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-@$#');
+-- Unexpected positional argument
 SELECT mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd');
+-- Duplicate parameter assignment
+SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', digitChar => 'e');
+-- Required parameter not found
+SELECT mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd');
+-- Unrecognized parameter name
+SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', cellular => 'automata');

Review Comment:
   Could you add one more test for `NAMED_ARGUMENTS_NOT_SUPPORTED`?



##########
sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql:
##########
@@ -1,5 +1,13 @@
+-- Test for named arguments
 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-@$#');
+-- Unexpected positional argument
 SELECT mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd');
+-- Duplicate parameter assignment
+SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', digitChar => 'e');

Review Comment:
   Could you add one more test for duplicated assignments on a positional arg and a named arg?



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2114,6 +2125,11 @@
     ],
     "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."
+    ]

Review Comment:
   IIRC new error classes better come with sqlState whenever possible. cc @srielau 



##########
sql/core/src/test/resources/sql-tests/inputs/mask-functions.sql:
##########
@@ -55,4 +55,4 @@ SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 'a') as t(c1, rep
 SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 'ABC') as t(c1, replaceArg);
 SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 123) as t(c1, replaceArg);
 SELECT mask('abcd-EFGH-8765-4321', 'A', 'w', '');
-SELECT mask('abcd-EFGH-8765-4321', 'A', 'abc');
\ No newline at end of file
+SELECT mask('abcd-EFGH-8765-4321', 'A', 'abc');

Review Comment:
   Could you revert the changes of this file?



##########
sql/core/src/test/scala/org/apache/spark/sql/CountMinSketchAggQuerySuite.scala:
##########
@@ -61,4 +61,23 @@ class CountMinSketchAggQuerySuite extends QueryTest with SharedSparkSession {
 
     assert(sketch == reference)
   }
+
+  test("count-min sketch with named-arguments") {

Review Comment:
   Shall we add some e2e tests on the function to `sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql`?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(name: String,

Review Comment:
   ```suggestion
   case class NamedArgument(
       name: String,
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonwang-db commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "allisonwang-db (via GitHub)" <gi...@apache.org>.
allisonwang-db commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1262800447


##########
sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out:
##########
@@ -327,13 +327,14 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
-  "sqlState" : "42605",
+  "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",

Review Comment:
   Agree. We shouldn't change the error messages for function invocations that do not use named arguments. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1253691138


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * A general trait which is used to identify the DataType of the argument
+ */
+trait NamedArgumentType
+
+/**
+ * The standard case class used to represent a simple data type
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * A named parameter
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+    dataType: NamedArgumentType,
+    default: Option[Expression] = None)
+
+/**
+ * Represents a method signature and the list of arguments it receives as input.
+ * Currently, overloads are not supported and only one FunctionSignature is allowed
+ * per function expression.
+ *
+ * @param parameters The list of arguments which the function takes
+ */
+case class FunctionSignature(parameters: Seq[NamedArgument])
+
+/**
+ * The class which companion objects of function expression implement to
+ * support named arguments for that function expression.
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use the provided version rearrange as it is consistent with
+   * the SQL standard. If absolutely necessary the developer can choose to override the default
+   * behavior for additional flexibility.
+   *
+   * @param functionSignature Function signature that denotes positional order of arguments
+   * @param args The sequence of expressions from function invocation
+   * @param functionName The name of the function invoked for debugging purposes
+   * @return positional order of arguments according to FunctionSignature
+   */
+  protected def rearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(functionSignature, args, functionName)
+  }
+}
+
+object SupportsNamedArguments {
+
+  /**
+   * Given a generic type, we check if the companion object of said type exists.
+   * If that object extends the trait [[SupportsNamedArguments]], then we rearrange
+   * the expressions in the order specified by the object.
+   *
+   * It is here we resubstitute [[Unevaluable]] [[NamedArgumentExpression]]s with
+   * normal expressions. This method will produce an positional argument list which
+   * is equivalent to the original argumnet list, except the expressions are now
+   * fit for consumption by [[ResolveFunctions]]
+   *
+   * @param expressions The list of positional and named argument expressions
+   * @tparam T The actual expression class.
+   * @return positional argument list
+   */
+  final def getRearrangedExpressions[T <: Expression : ClassTag](
+      expressions: Seq[Expression], functionName: String): Seq[Expression] = {
+
+    if (!expressions.exists(_.isInstanceOf[NamedArgumentExpression])) {
+      return expressions
+    }
+
+    import scala.reflect.runtime.currentMirror
+
+    // This code heavily utilizes Scala reflection which is unfamiliar to most developers.
+    // Here are the steps of this function:
+    // 1. Obtain the module symbol for the companion object of the function expression.
+    // 2. Obtain the module class symbol that represents the companion object.
+    // 3. Check if the base classes of the module class symbol contains SupportsNamedArguments.
+    //    This checks if the companion object is an implementor of SupportsNamedArguments.
+    // 4. Check if the module class symbol is a top level object. Reflection is unable to
+    //    obtain a companion object instance if it is member of some enclosing class unless
+    //    instance of said enclosing class is provided which we do not have.
+    // 5. Use reflection to obtain instance of companion object and perform immediate cast to
+    //    SupportsNamedArguments as it is already verified the cast is safe.
+    // 6. Obtain function signature and rearrange expression according to the given signature.
+    val runtimeClass = scala.reflect.classTag[T].runtimeClass
+    val targetModuleSymbol = currentMirror.classSymbol(runtimeClass).companion
+    val parentClass = scala.reflect.classTag[SupportsNamedArguments].runtimeClass
+    val parentSymbol = currentMirror.classSymbol(parentClass)
+
+    targetModuleSymbol match {
+      case scala.reflect.runtime.universe.NoSymbol =>

Review Comment:
   Then you can just compare for equality using == ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on PR #41864:
URL: https://github.com/apache/spark/pull/41864#issuecomment-1622012293

   @dtenedor @anchovYu Can you leave some review comments on this new PR! I rebased it and it should be in a more compact form ready for review! 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dtenedor commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1262838682


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -980,6 +1014,20 @@ object TableFunctionRegistry {
     (name, (info, (expressions: Seq[Expression]) => builder(expressions)))
   }
 
+  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)
+      val plan = builder.build(name, rearrangedExpressions)

Review Comment:
   you don't need the `val plan` here since you can just return `builder.build(...)`



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -1022,6 +1070,68 @@ object TableFunctionRegistry {
   val functionSet: Set[FunctionIdentifier] = builtin.listFunction().toSet
 }
 
-trait ExpressionBuilder {
-  def build(funcName: String, expressions: Seq[Expression]): Expression
+trait Builder[T] {

Review Comment:
   please put some kind of short general comment for this `Builder` class mentioning that it is an interface for describing the expected parameters for SQL functions and takes responsibility for validating provided arguments for tham.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -1022,6 +1070,68 @@ object TableFunctionRegistry {
   val functionSet: Set[FunctionIdentifier] = builtin.listFunction().toSet
 }
 
-trait ExpressionBuilder {
-  def build(funcName: String, expressions: Seq[Expression]): Expression
+trait Builder[T] {
+  /**
+   * A method that returns the signatures of overloads that are associated with this function.
+   * Each function signature includes a list of parameters to which the analyzer can
+   * compare a function call with provided arguments to determine if that function
+   * call is a match for the function signature.
+   *
+   * @return a list of function signatures
+   */
+  def functionSignatures: Option[Seq[FunctionSignature]] = None
+
+  /**
+   * This function rearranges the arguments provided during function invocation in positional order
+   * according to the function signature. This method will fill in the default values if optional
+   * parameters do not have their values specified. Any function which supports named arguments
+   * will have this routine invoked, even if no named arguments are present in the argument list.
+   * This is done to eliminate constructor overloads in some methods which use them for default
+   * values prior to the implementation of the named argument framework. This function will also
+   * check if the number of arguments are correct. If that is not the case, then an error will be
+   * thrown.
+   *
+   * IMPORTANT: This method will be called before the [[Builder.build]] method is invoked. It is
+   * guaranteed that the expressions provided to the [[Builder.build]] functions forms a valid set
+   * of argument expressions that can be used in the construction of the function expression.
+   *
+   * @param expectedSignature The method signature which we rearrange our arguments according to
+   * @param providedArguments The list of arguments passed from function invocation
+   * @param functionName The name of the function
+   * @return The rearranged arugument list with arguments in positional order
+   */
+  def rearrange(
+      expectedSignature: FunctionSignature,
+      providedArguments: Seq[Expression],
+      functionName: String) : Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(expectedSignature, providedArguments, functionName)
+  }
+
+  def build(funcName: String, expressions: Seq[Expression]): T
+}
+
+/**
+ * A trait used for scalar valued functions that defines how their expression representations

Review Comment:
   please change this to a complete sentence ending with a period (same for L1120 below)



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -50,6 +50,90 @@ import org.apache.spark.sql.types._
  */
 private[sql] object QueryCompilationErrors extends QueryErrorsBase {
 
+  def unexpectedRequiredParameterInFunctionSignature(
+      functionName: String, functionSignature: FunctionSignature) : Throwable = {
+    val errorMessage = s"Function $functionName has an unexpected required argument for" +
+      s" the provided function signature $functionSignature. All required arguments should" +
+      s" come before optional arguments."
+    SparkException.internalError(errorMessage)
+  }
+
+  def multipleFunctionSignatures(functionName: String,
+      functionSignatures: Seq[FunctionSignature]): Throwable = {
+    var errorMessage = s"Function $functionName cannot have multiple method signatures." +
+      s" The function signatures found were: \n"
+    for (functionSignature <- functionSignatures) {
+      errorMessage = errorMessage + s"${functionSignature}\n"
+    }
+    SparkException.internalError(errorMessage)
+  }
+
+  def namedArgumentsNotSupported(functionName: String) : Throwable = {
+    new AnalysisException(
+      errorClass = "NAMED_ARGUMENTS_NOT_SUPPORTED",
+      messageParameters = Map("functionName" -> toSQLId(functionName))
+    )
+  }
+
+  def positionalAndNamedArgumentDoubleReference(
+      functionName: String, parameterName: String) : Throwable = {
+    val errorClass =
+      "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.POSITIONAL_AND_NAMED_ARGUMENT_DOUBLE_REFERENCE"

Review Comment:
   maybe `DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.BOTH_POSITIONAL_AND_NAMED`?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -887,13 +887,46 @@ 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 <: Builder[_]](
+      name: String,
+      builder: T,
+      expressions: Seq[Expression]) : Seq[Expression] = {
+    val rearrangedExpressions = if (!builder.functionSignatures.isEmpty) {
+      val functionSignatures = builder.functionSignatures.get
+      if (functionSignatures.length != 1) {

Review Comment:
   Good question, I imagine we probably would not get around to prioritizing overloads soon with this framework. But if we did, would it become necessary to update all the places where these function signatures are defined, from an `Option[FunctionSignature]` to an `Option[Seq[FunctionSignature]]`? In contrast, if we go with the latter syntax of a `Seq` of just one function signature, we wouldn't have to update all those sites later.
   
   However, I imagine such an update would be a fairly mechanical change, and in the interim, we get better compiler enforcement. It seems like this suggestion is better for now to switch the API to return just an `Option[FunctionSignature]` for now.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -980,6 +1014,20 @@ object TableFunctionRegistry {
     (name, (info, (expressions: Seq[Expression]) => builder(expressions)))
   }
 
+  def generatorBuilder[T <: GeneratorBuilder : ClassTag](

Review Comment:
   please add a comment for this method



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/FunctionBuilderBase.scala:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+trait FunctionBuilderBase[T] {
+  /**
+   * A method that returns the signatures of overloads that are associated with this function.
+   * Each function signature includes a list of parameters to which the analyzer can
+   * compare a function call with provided arguments to determine if that function
+   * call is a match for the function signature.
+   *
+   * @return a list of function signatures
+   */
+  def functionSignatures: Option[Seq[FunctionSignature]] = None
+
+  /**
+   * This function rearranges the arguments provided during function invocation in positional order
+   * according to the function signature. This method will fill in the default values if optional
+   * parameters do not have their values specified. Any function which supports named arguments
+   * will have this routine invoked, even if no named arguments are present in the argument list.
+   * This is done to eliminate constructor overloads in some methods which use them for default
+   * values prior to the implementation of the named argument framework. This function will also
+   * check if the number of arguments are correct. If that is not the case, then an error will be
+   * thrown.
+   *
+   * IMPORTANT: This method will be called before the [[FunctionBuilderBase.build]] method is
+   * invoked. It is guaranteed that the expressions provided to the [[FunctionBuilderBase.build]]
+   * functions forms a valid set of argument expressions that can be used in the construction of
+   * the function expression.
+   *
+   * @param expectedSignature The method signature which we rearrange our arguments according to
+   * @param providedArguments The list of arguments passed from function invocation
+   * @param functionName The name of the function
+   * @return The rearranged argument list with arguments in positional order
+   */
+  def rearrange(
+                 expectedSignature: FunctionSignature,
+                 providedArguments: Seq[Expression],
+                 functionName: String) : Seq[Expression] = {
+    NamedArgumentsSupport.defaultRearrange(expectedSignature, providedArguments, functionName)
+  }
+
+  def build(funcName: String, expressions: Seq[Expression]): T
+}
+
+object NamedArgumentsSupport {
+  /**
+   * This method is the default routine which rearranges the arguments in positional order according
+   * to the function signature provided. This will also fill in any default values that exists for
+   * optional arguments. This method will also be invoked even if there are no named arguments in
+   * the argument list.
+   *
+   * @param functionSignature The function signature that defines the positional ordering
+   * @param args The argument list provided in function invocation
+   * @param functionName The name of the function
+   * @return A list of arguments rearranged in positional order defined by the provided signature
+   */
+  final def defaultRearrange(
+      functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters: Seq[NamedArgument] = functionSignature.parameters
+    if (parameters.dropWhile(_.default.isEmpty).exists(_.default.isEmpty)) {
+      throw QueryCompilationErrors.unexpectedRequiredParameterInFunctionSignature(
+        functionName, functionSignature)
+    }
+
+    val (positionalArgs, namedArgs) = args.span(!_.isInstanceOf[NamedArgumentExpression])
+    val namedParameters: Seq[NamedArgument] = parameters.drop(positionalArgs.size)
+
+    // The following loop checks for the following:
+    // 1. Unrecognized parameter names
+    // 2. Duplicate routine parameter assignments
+    val allParameterNames: Seq[String] = parameters.map(_.name)
+    val parameterNamesSet: Set[String] = allParameterNames.toSet
+    val positionalParametersSet = allParameterNames.take(positionalArgs.size).toSet
+    val namedParametersSet = collection.mutable.Set[String]()
+
+    for (arg <- namedArgs) {
+      arg match {
+        case namedArg: NamedArgumentExpression =>
+          val parameterName = namedArg.key
+          if (!parameterNamesSet.contains(parameterName)) {
+            throw QueryCompilationErrors.unrecognizedParameterName(functionName, namedArg.key,
+              parameterNamesSet.toSeq)
+          }
+          if (positionalParametersSet.contains(parameterName)) {
+            throw QueryCompilationErrors.positionalAndNamedArgumentDoubleReference(
+              functionName, namedArg.key)
+          }
+          if (namedParametersSet.contains(parameterName)) {
+            throw QueryCompilationErrors.doubleNamedArgumentReference(
+              functionName, namedArg.key)
+          }
+          namedParametersSet.add(namedArg.key)
+        case _ =>
+          throw QueryCompilationErrors.unexpectedPositionalArgument(functionName)
+      }
+    }
+
+    // Check argument list size against provided parameter list length

Review Comment:
   ```suggestion
       // Check the argument list size against the provided parameter list length.
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/FunctionBuilderBase.scala:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+trait FunctionBuilderBase[T] {
+  /**
+   * A method that returns the signatures of overloads that are associated with this function.
+   * Each function signature includes a list of parameters to which the analyzer can
+   * compare a function call with provided arguments to determine if that function
+   * call is a match for the function signature.
+   *
+   * @return a list of function signatures

Review Comment:
   no need for this line since the above comment conveys that information.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/FunctionBuilderBase.scala:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+trait FunctionBuilderBase[T] {

Review Comment:
   please add a general comment describing what this object represents (using full sentences). Same for `NamedArgumentsSupport` below.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] anchovYu commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "anchovYu (via GitHub)" <gi...@apache.org>.
anchovYu commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1262848538


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/FunctionBuilderBase.scala:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+trait FunctionBuilderBase[T] {
+  /**
+   * A method that returns the signatures of overloads that are associated with this function.
+   * Each function signature includes a list of parameters to which the analyzer can
+   * compare a function call with provided arguments to determine if that function
+   * call is a match for the function signature.
+   *

Review Comment:
   Could you add comment that we only support 1) one function signature 2) required should come before optional?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/FunctionBuilderBase.scala:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+trait FunctionBuilderBase[T] {
+  /**
+   * A method that returns the signatures of overloads that are associated with this function.
+   * Each function signature includes a list of parameters to which the analyzer can
+   * compare a function call with provided arguments to determine if that function
+   * call is a match for the function signature.
+   *
+   * @return a list of function signatures
+   */
+  def functionSignatures: Option[Seq[FunctionSignature]] = None
+
+  /**
+   * This function rearranges the arguments provided during function invocation in positional order
+   * according to the function signature. This method will fill in the default values if optional
+   * parameters do not have their values specified. Any function which supports named arguments
+   * will have this routine invoked, even if no named arguments are present in the argument list.
+   * This is done to eliminate constructor overloads in some methods which use them for default
+   * values prior to the implementation of the named argument framework. This function will also
+   * check if the number of arguments are correct. If that is not the case, then an error will be
+   * thrown.
+   *
+   * IMPORTANT: This method will be called before the [[FunctionBuilderBase.build]] method is
+   * invoked. It is guaranteed that the expressions provided to the [[FunctionBuilderBase.build]]
+   * functions forms a valid set of argument expressions that can be used in the construction of
+   * the function expression.
+   *
+   * @param expectedSignature The method signature which we rearrange our arguments according to
+   * @param providedArguments The list of arguments passed from function invocation
+   * @param functionName The name of the function
+   * @return The rearranged argument list with arguments in positional order
+   */
+  def rearrange(
+                 expectedSignature: FunctionSignature,
+                 providedArguments: Seq[Expression],
+                 functionName: String) : Seq[Expression] = {
+    NamedArgumentsSupport.defaultRearrange(expectedSignature, providedArguments, functionName)
+  }
+
+  def build(funcName: String, expressions: Seq[Expression]): T
+}
+
+object NamedArgumentsSupport {
+  /**
+   * This method is the default routine which rearranges the arguments in positional order according
+   * to the function signature provided. This will also fill in any default values that exists for
+   * optional arguments. This method will also be invoked even if there are no named arguments in
+   * the argument list.
+   *
+   * @param functionSignature The function signature that defines the positional ordering
+   * @param args The argument list provided in function invocation
+   * @param functionName The name of the function
+   * @return A list of arguments rearranged in positional order defined by the provided signature
+   */
+  final def defaultRearrange(
+      functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters: Seq[NamedArgument] = functionSignature.parameters
+    if (parameters.dropWhile(_.default.isEmpty).exists(_.default.isEmpty)) {
+      throw QueryCompilationErrors.unexpectedRequiredParameterInFunctionSignature(
+        functionName, functionSignature)
+    }
+
+    val (positionalArgs, namedArgs) = args.span(!_.isInstanceOf[NamedArgumentExpression])
+    val namedParameters: Seq[NamedArgument] = parameters.drop(positionalArgs.size)
+
+    // The following loop checks for the following:
+    // 1. Unrecognized parameter names
+    // 2. Duplicate routine parameter assignments
+    val allParameterNames: Seq[String] = parameters.map(_.name)
+    val parameterNamesSet: Set[String] = allParameterNames.toSet
+    val positionalParametersSet = allParameterNames.take(positionalArgs.size).toSet
+    val namedParametersSet = collection.mutable.Set[String]()
+
+    for (arg <- namedArgs) {
+      arg match {
+        case namedArg: NamedArgumentExpression =>
+          val parameterName = namedArg.key
+          if (!parameterNamesSet.contains(parameterName)) {
+            throw QueryCompilationErrors.unrecognizedParameterName(functionName, namedArg.key,
+              parameterNamesSet.toSeq)
+          }
+          if (positionalParametersSet.contains(parameterName)) {
+            throw QueryCompilationErrors.positionalAndNamedArgumentDoubleReference(
+              functionName, namedArg.key)
+          }
+          if (namedParametersSet.contains(parameterName)) {
+            throw QueryCompilationErrors.doubleNamedArgumentReference(
+              functionName, namedArg.key)
+          }
+          namedParametersSet.add(namedArg.key)
+        case _ =>
+          throw QueryCompilationErrors.unexpectedPositionalArgument(functionName)
+      }
+    }
+
+    // Check argument list size against provided parameter list length
+    if (parameters.size < args.length) {
+      val validParameterSizes =
+        Array.range(parameters.count(_.default.isEmpty), parameters.size + 1).toSeq
+      throw QueryCompilationErrors.wrongNumArgsError(
+        functionName, validParameterSizes, args.length)
+    }
+
+    // This constructs a map from argument name to value for argument rearrangement.
+    val namedArgMap = namedArgs.map { arg =>
+      val namedArg = arg.asInstanceOf[NamedArgumentExpression]
+      namedArg.key -> namedArg.value
+    }.toMap
+
+    // We rearrange named arguments to match their positional order.
+    val rearrangedNamedArgs: Seq[Expression] = namedParameters.map { param =>
+      namedArgMap.getOrElse(
+        param.name,
+        if (param.default.isEmpty) {
+          throw QueryCompilationErrors.requiredParameterNotFound(functionName, param.name)
+        } else {
+          param.default.get
+        }
+      )
+    }
+    positionalArgs ++ rearrangedNamedArgs

Review Comment:
   Could you add an assertion that the size of returning results is equal to the parameter count in function signature? Meanwhile, `build` can also have similar assertion as well.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -935,9 +969,9 @@ 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)

Review Comment:
   It this change necessary?



##########
sql/core/src/test/resources/sql-tests/inputs/named-function-arguments.sql:
##########
@@ -1,5 +1,50 @@
+-- Test for named arguments for Mask

Review Comment:
   Could you add more tests on passing in all positional args and filling in the defaults?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/FunctionBuilderBase.scala:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+trait FunctionBuilderBase[T] {
+  /**
+   * A method that returns the signatures of overloads that are associated with this function.
+   * Each function signature includes a list of parameters to which the analyzer can
+   * compare a function call with provided arguments to determine if that function
+   * call is a match for the function signature.
+   *
+   * @return a list of function signatures
+   */
+  def functionSignatures: Option[Seq[FunctionSignature]] = None
+
+  /**
+   * This function rearranges the arguments provided during function invocation in positional order
+   * according to the function signature. This method will fill in the default values if optional
+   * parameters do not have their values specified. Any function which supports named arguments
+   * will have this routine invoked, even if no named arguments are present in the argument list.
+   * This is done to eliminate constructor overloads in some methods which use them for default
+   * values prior to the implementation of the named argument framework. This function will also
+   * check if the number of arguments are correct. If that is not the case, then an error will be
+   * thrown.
+   *
+   * IMPORTANT: This method will be called before the [[FunctionBuilderBase.build]] method is
+   * invoked. It is guaranteed that the expressions provided to the [[FunctionBuilderBase.build]]
+   * functions forms a valid set of argument expressions that can be used in the construction of
+   * the function expression.
+   *
+   * @param expectedSignature The method signature which we rearrange our arguments according to
+   * @param providedArguments The list of arguments passed from function invocation
+   * @param functionName The name of the function
+   * @return The rearranged argument list with arguments in positional order
+   */
+  def rearrange(
+                 expectedSignature: FunctionSignature,
+                 providedArguments: Seq[Expression],
+                 functionName: String) : Seq[Expression] = {
+    NamedArgumentsSupport.defaultRearrange(expectedSignature, providedArguments, functionName)
+  }
+
+  def build(funcName: String, expressions: Seq[Expression]): T
+}
+
+object NamedArgumentsSupport {
+  /**
+   * This method is the default routine which rearranges the arguments in positional order according
+   * to the function signature provided. This will also fill in any default values that exists for
+   * optional arguments. This method will also be invoked even if there are no named arguments in
+   * the argument list.
+   *
+   * @param functionSignature The function signature that defines the positional ordering
+   * @param args The argument list provided in function invocation
+   * @param functionName The name of the function
+   * @return A list of arguments rearranged in positional order defined by the provided signature
+   */
+  final def defaultRearrange(
+      functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters: Seq[NamedArgument] = functionSignature.parameters
+    if (parameters.dropWhile(_.default.isEmpty).exists(_.default.isEmpty)) {
+      throw QueryCompilationErrors.unexpectedRequiredParameterInFunctionSignature(
+        functionName, functionSignature)
+    }
+
+    val (positionalArgs, namedArgs) = args.span(!_.isInstanceOf[NamedArgumentExpression])
+    val namedParameters: Seq[NamedArgument] = parameters.drop(positionalArgs.size)
+
+    // The following loop checks for the following:
+    // 1. Unrecognized parameter names
+    // 2. Duplicate routine parameter assignments
+    val allParameterNames: Seq[String] = parameters.map(_.name)
+    val parameterNamesSet: Set[String] = allParameterNames.toSet
+    val positionalParametersSet = allParameterNames.take(positionalArgs.size).toSet
+    val namedParametersSet = collection.mutable.Set[String]()
+
+    for (arg <- namedArgs) {
+      arg match {
+        case namedArg: NamedArgumentExpression =>
+          val parameterName = namedArg.key
+          if (!parameterNamesSet.contains(parameterName)) {
+            throw QueryCompilationErrors.unrecognizedParameterName(functionName, namedArg.key,
+              parameterNamesSet.toSeq)
+          }
+          if (positionalParametersSet.contains(parameterName)) {
+            throw QueryCompilationErrors.positionalAndNamedArgumentDoubleReference(
+              functionName, namedArg.key)
+          }
+          if (namedParametersSet.contains(parameterName)) {
+            throw QueryCompilationErrors.doubleNamedArgumentReference(
+              functionName, namedArg.key)
+          }
+          namedParametersSet.add(namedArg.key)
+        case _ =>
+          throw QueryCompilationErrors.unexpectedPositionalArgument(functionName)
+      }
+    }
+
+    // Check argument list size against provided parameter list length
+    if (parameters.size < args.length) {

Review Comment:
   Could you add tests for this case? Is this exception only possible to hit when passing in all positional args?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/FunctionBuilderBase.scala:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+trait FunctionBuilderBase[T] {
+  /**
+   * A method that returns the signatures of overloads that are associated with this function.
+   * Each function signature includes a list of parameters to which the analyzer can
+   * compare a function call with provided arguments to determine if that function
+   * call is a match for the function signature.
+   *
+   * @return a list of function signatures
+   */
+  def functionSignatures: Option[Seq[FunctionSignature]] = None
+
+  /**
+   * This function rearranges the arguments provided during function invocation in positional order
+   * according to the function signature. This method will fill in the default values if optional
+   * parameters do not have their values specified. Any function which supports named arguments
+   * will have this routine invoked, even if no named arguments are present in the argument list.
+   * This is done to eliminate constructor overloads in some methods which use them for default
+   * values prior to the implementation of the named argument framework. This function will also
+   * check if the number of arguments are correct. If that is not the case, then an error will be
+   * thrown.
+   *
+   * IMPORTANT: This method will be called before the [[FunctionBuilderBase.build]] method is
+   * invoked. It is guaranteed that the expressions provided to the [[FunctionBuilderBase.build]]
+   * functions forms a valid set of argument expressions that can be used in the construction of
+   * the function expression.
+   *
+   * @param expectedSignature The method signature which we rearrange our arguments according to
+   * @param providedArguments The list of arguments passed from function invocation
+   * @param functionName The name of the function
+   * @return The rearranged argument list with arguments in positional order
+   */
+  def rearrange(
+                 expectedSignature: FunctionSignature,
+                 providedArguments: Seq[Expression],
+                 functionName: String) : Seq[Expression] = {
+    NamedArgumentsSupport.defaultRearrange(expectedSignature, providedArguments, functionName)
+  }
+
+  def build(funcName: String, expressions: Seq[Expression]): T
+}
+
+object NamedArgumentsSupport {
+  /**
+   * This method is the default routine which rearranges the arguments in positional order according
+   * to the function signature provided. This will also fill in any default values that exists for
+   * optional arguments. This method will also be invoked even if there are no named arguments in
+   * the argument list.

Review Comment:
   Could you add more details:
   1. keeps all positional args in the original order
   2. rearranges named arguments according to the order defined in function signature, fill in missing optional arguments with default values



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] srielau commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "srielau (via GitHub)" <gi...@apache.org>.
srielau commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1263145242


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -1893,6 +1911,12 @@
       "Not allowed to implement multiple UDF interfaces, UDF class <className>."
     ]
   },
+  "NAMED_ARGUMENTS_NOT_SUPPORTED" : {
+    "message" : [
+      "Named arguments are not supported for function <functionName>; please retry the query with positional arguments to the function call instead."

Review Comment:
   ```suggestion
     "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."
   ```



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -1893,6 +1911,12 @@
       "Not allowed to implement multiple UDF interfaces, UDF class <className>."
     ]
   },
+  "NAMED_ARGUMENTS_NOT_SUPPORTED" : {
+    "message" : [
+      "Named arguments are not supported for function <functionName>; please retry the query with positional arguments to the function call instead."
+    ],
+    "sqlState" : "4274K"
+  },
   "NAMED_ARGUMENTS_SUPPORT_DISABLED" : {

Review Comment:
   ```suggestion
     "NAMED_PARAMETER_SUPPORT_DISABLED" : {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 closed pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 closed pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions
URL: https://github.com/apache/spark/pull/41864


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on PR #41864:
URL: https://github.com/apache/spark/pull/41864#issuecomment-1622280813

   @anchovYu Comments addressed!
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] anchovYu commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "anchovYu (via GitHub)" <gi...@apache.org>.
anchovYu commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1254897374


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -894,8 +896,10 @@ object FunctionRegistry {
       since: Option[String] = None): (String, (ExpressionInfo, FunctionBuilder)) = {
     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 =
+        SupportsNamedArguments.getRearrangedExpressions[T](expressions, name)

Review Comment:
   Does it work? Most `ExpressionBuilder` is an object, e.g. `CeilExpressionBuilder`, `FloorExpressionBuilder`. Does the `SupportsNamedArguments.getRearrangedExpressions` code work when T is an object?
   
   Actually I think we can leave out the support for expression builder in this PR for now. Most expression builder is used only when it has complicated initializer, e.g. build with one param and build with two params. That may require at least two function signatures, or a refactor on the existing code to make it use requires one signature.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] anchovYu commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "anchovYu (via GitHub)" <gi...@apache.org>.
anchovYu commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1254897374


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -894,8 +896,10 @@ object FunctionRegistry {
       since: Option[String] = None): (String, (ExpressionInfo, FunctionBuilder)) = {
     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 =
+        SupportsNamedArguments.getRearrangedExpressions[T](expressions, name)

Review Comment:
   Does it work? Most `ExpressionBuilder` is an object, e.g. `CeilExpressionBuilder`, `FloorExpressionBuilder`. Does the `SupportsNamedArguments.getRearrangedExpressions` code work when T is an object?
   
   Actually I think we can leave out the support for expression builder in this PR for now. Most expression builder is used only when it has complicated initializer, e.g. build with one param and build with two params. That may require at least two function signatures, or a refactor on the existing code to make it use one signature.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] anchovYu commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "anchovYu (via GitHub)" <gi...@apache.org>.
anchovYu commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1254717223


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * The class which companion objects of function expression may implement to
+ * support named arguments for that function expression.
+ *
+ * Example:
+ *  object CountMinSketchAgg extends SupportsNamedArguments {
+ *    final val functionSignature = FunctionSignature(Seq(
+ *      NamedArgument("column",
+ *          FixedArgumentType(TypeCollection(IntegralType, StringType, BinaryType))),
+ *      NamedArgument("epsilon", FixedArgumentType(DoubleType)),
+ *      NamedArgument("confidence", FixedArgumentType(DoubleType)),
+ *      NamedArgument("seed", FixedArgumentType(IntegerType))
+ *    ))
+ *    override def functionSignatures: Seq[FunctionSignature] = Seq(functionSignature)
+ *  }
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use this provided implementation as it is consistent with
+   * the SQL standard. If absolutely necessary the developer can choose to override the default
+   * behavior for additional flexibility.
+   *
+   * @param expectedSignature Function signature that denotes positional order of arguments
+   * @param providedArguments The sequence of expressions from function invocation
+   * @param functionName The name of the function invoked for debugging purposes
+   * @return positional order of arguments according to FunctionSignature obtained
+   *         by changing the order of the above provided arguments
+   */
+  protected def rearrange(
+      expectedSignature: FunctionSignature,
+      providedArguments: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(expectedSignature, providedArguments, functionName)
+  }
+}
+
+object SupportsNamedArguments {
+
+  /**
+   * Given a generic type, we check if the companion object of said type exists.
+   * If that object extends the trait [[SupportsNamedArguments]], then we rearrange
+   * the expressions in the order specified by the object.
+   *
+   * It is here we resubstitute [[Unevaluable]] [[NamedArgumentExpression]]s with
+   * normal expressions. This method will produce an positional argument list which
+   * is equivalent to the original argumnet list, except the expressions are now
+   * fit for consumption by [[ResolveFunctions]]
+   *
+   * @param expressions The list of positional and named argument expressions
+   * @tparam T The actual expression class.
+   * @return positional argument list
+   */
+  final def getRearrangedExpressions[T <: Expression : ClassTag](
+      expressions: Seq[Expression], functionName: String): Seq[Expression] = {
+
+    if (!expressions.exists(_.isInstanceOf[NamedArgumentExpression])) {
+      return expressions
+    }
+
+    import scala.reflect.runtime.currentMirror
+
+    // This code heavily utilizes Scala reflection which is unfamiliar to most developers.
+    // Here are the steps of this function:
+    // 1. Obtain the module symbol for the companion object of the function expression.
+    // 2. Obtain the module class symbol that represents the companion object.
+    // 3. Check if the base classes of the module class symbol contains SupportsNamedArguments.
+    //    This checks if the companion object is an implementor of SupportsNamedArguments.
+    // 4. Check if the module class symbol is a top level object. Reflection is unable to
+    //    obtain a companion object instance if it is member of some enclosing class unless
+    //    instance of said enclosing class is provided which we do not have.
+    // 5. Use reflection to obtain instance of companion object and perform immediate cast to
+    //    SupportsNamedArguments as it is already verified the cast is safe.
+    // 6. Obtain function signature and rearrange expression according to the given signature.
+    val runtimeClass = scala.reflect.classTag[T].runtimeClass
+    val targetModuleSymbol = currentMirror.classSymbol(runtimeClass).companion
+    val parentClass = scala.reflect.classTag[SupportsNamedArguments].runtimeClass
+    val parentSymbol = currentMirror.classSymbol(parentClass)
+
+    if(targetModuleSymbol == scala.reflect.runtime.universe.NoSymbol) {
+      throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
+    }
+
+    val moduleClassSymbol = targetModuleSymbol.asModule.moduleClass.asClass
+    if (!moduleClassSymbol.baseClasses.contains(parentSymbol)) {
+      throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
+    }
+    if (currentMirror.runtimeClass(moduleClassSymbol).getEnclosingClass != null) {
+      throw QueryCompilationErrors.cannotObtainCompanionObjectInstance(functionName)
+    }
+    val instance = currentMirror.reflectModule(targetModuleSymbol.asModule)
+      .instance.asInstanceOf[SupportsNamedArguments]
+    if (instance.functionSignatures.size != 1) {
+      throw QueryCompilationErrors.multipleFunctionSignatures(
+        functionName, instance.functionSignatures)
+    }
+    instance.rearrange(instance.functionSignatures.head, expressions, functionName)
+  }
+
+  final def defaultRearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters: Seq[NamedArgument] = functionSignature.parameters
+    val firstNamedArgIdx: Int = args.indexWhere(_.isInstanceOf[NamedArgumentExpression])
+    val (positionalArgs, namedArgs) = args.splitAt(firstNamedArgIdx)
+    val namedParameters: Seq[NamedArgument] = parameters.drop(positionalArgs.size)
+
+    // Performing some checking to ensure valid argument list
+    val allParameterNames: Seq[String] = parameters.map(_.name)
+    val parameterNamesSet: Set[String] = allParameterNames.toSet
+    val assignedParameterSet = collection.mutable.Set[String](
+      allParameterNames.take(positionalArgs.size): _*)
+    for (arg <- namedArgs) {
+      arg match {
+        case namedArg: NamedArgumentExpression =>
+          if (assignedParameterSet.contains(namedArg.key)) {
+            throw QueryCompilationErrors.duplicateRoutineParameterAssignment(
+              functionName, namedArg.key)
+          }
+          if (!parameterNamesSet.contains(namedArg.key)) {
+            throw QueryCompilationErrors.unrecognizedParameterName(functionName, namedArg.key,
+              parameterNamesSet.toSeq)
+          }
+          assignedParameterSet.add(namedArg.key)
+        case _ =>
+          throw QueryCompilationErrors.unexpectedPositionalArgument(functionName)
+      }
+    }
+
+    // Construct a map from argument name to value for argument rearrangement
+    val namedArgMap = namedArgs.map { arg =>
+      val namedArg = arg.asInstanceOf[NamedArgumentExpression]
+      namedArg.key -> namedArg.value
+    }.toMap
+
+    // Rearrange named arguments to match their positional order
+    val rearrangedNamedArgs: Seq[Expression] = namedParameters.map { param =>
+      namedArgMap.getOrElse(
+        param.name,
+        if (param.default.isEmpty) {
+          throw QueryCompilationErrors.requiredParameterNotFound(functionName, param.name)
+        } else {
+          param.default.get
+        }
+      )
+    }
+    positionalArgs ++ rearrangedNamedArgs
+  }
+}
+
+/**
+ * Identifies which forms of provided argument values are expected for each call
+ * to the associated SQL function
+ */
+trait NamedArgumentType
+
+/**
+ * Represents a named argument that expects a scalar value of one specific DataType
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * Represents a parameter of a function expression. Function expressions should use this class
+ * to construct the argument lists returned in [[SupportsNamedArguments.functionSignatures]]
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+                          dataType: NamedArgumentType,

Review Comment:
   All parameters should have 4 space indentation.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1253551872


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2333,6 +2354,11 @@
     ],
     "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."

Review Comment:
   Could you help users and propose function parameters sorted by levenshtein distance? See for instance:
   https://github.com/apache/spark/blob/d9c2d38f14a400ea078492d770b749f506a8401c/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala#L80



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -1781,6 +1787,11 @@
       "Not allowed to implement multiple UDF interfaces, UDF class <className>."
     ]
   },
+  "NAMED_ARGUMENTS_NOT_SUPPORTED" : {
+    "message" : [
+      "Named arguments are not supported for function <functionName>; please retry the query with positional arguments to the function call instead."

Review Comment:
   or enable the feature using the SQL config ...?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1254755937


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SupportsNamedArguments.scala:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.plans.logical
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.AbstractDataType
+
+/**
+ * The class which companion objects of function expression may implement to
+ * support named arguments for that function expression.
+ *
+ * Example:
+ *  object CountMinSketchAgg extends SupportsNamedArguments {
+ *    final val functionSignature = FunctionSignature(Seq(
+ *      NamedArgument("column",
+ *          FixedArgumentType(TypeCollection(IntegralType, StringType, BinaryType))),
+ *      NamedArgument("epsilon", FixedArgumentType(DoubleType)),
+ *      NamedArgument("confidence", FixedArgumentType(DoubleType)),
+ *      NamedArgument("seed", FixedArgumentType(IntegerType))
+ *    ))
+ *    override def functionSignatures: Seq[FunctionSignature] = Seq(functionSignature)
+ *  }
+ */
+abstract class SupportsNamedArguments {
+  /**
+   * This is the method overridden by function expressions to define their method signatures.
+   * Currently, we don't support overloads, so we restrict each function expression to return
+   * only one FunctionSignature.
+   *
+   * @return the signature of the function expression
+   */
+  def functionSignatures: Seq[FunctionSignature]
+
+  /**
+   * This function rearranges the list of expressions according to the function signature
+   * It is recommended to use this provided implementation as it is consistent with
+   * the SQL standard. If absolutely necessary the developer can choose to override the default
+   * behavior for additional flexibility.
+   *
+   * @param expectedSignature Function signature that denotes positional order of arguments
+   * @param providedArguments The sequence of expressions from function invocation
+   * @param functionName The name of the function invoked for debugging purposes
+   * @return positional order of arguments according to FunctionSignature obtained
+   *         by changing the order of the above provided arguments
+   */
+  protected def rearrange(
+      expectedSignature: FunctionSignature,
+      providedArguments: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    SupportsNamedArguments.defaultRearrange(expectedSignature, providedArguments, functionName)
+  }
+}
+
+object SupportsNamedArguments {
+
+  /**
+   * Given a generic type, we check if the companion object of said type exists.
+   * If that object extends the trait [[SupportsNamedArguments]], then we rearrange
+   * the expressions in the order specified by the object.
+   *
+   * It is here we resubstitute [[Unevaluable]] [[NamedArgumentExpression]]s with
+   * normal expressions. This method will produce an positional argument list which
+   * is equivalent to the original argumnet list, except the expressions are now
+   * fit for consumption by [[ResolveFunctions]]
+   *
+   * @param expressions The list of positional and named argument expressions
+   * @tparam T The actual expression class.
+   * @return positional argument list
+   */
+  final def getRearrangedExpressions[T <: Expression : ClassTag](
+      expressions: Seq[Expression], functionName: String): Seq[Expression] = {
+
+    if (!expressions.exists(_.isInstanceOf[NamedArgumentExpression])) {
+      return expressions
+    }
+
+    import scala.reflect.runtime.currentMirror
+
+    // This code heavily utilizes Scala reflection which is unfamiliar to most developers.
+    // Here are the steps of this function:
+    // 1. Obtain the module symbol for the companion object of the function expression.
+    // 2. Obtain the module class symbol that represents the companion object.
+    // 3. Check if the base classes of the module class symbol contains SupportsNamedArguments.
+    //    This checks if the companion object is an implementor of SupportsNamedArguments.
+    // 4. Check if the module class symbol is a top level object. Reflection is unable to
+    //    obtain a companion object instance if it is member of some enclosing class unless
+    //    instance of said enclosing class is provided which we do not have.
+    // 5. Use reflection to obtain instance of companion object and perform immediate cast to
+    //    SupportsNamedArguments as it is already verified the cast is safe.
+    // 6. Obtain function signature and rearrange expression according to the given signature.
+    val runtimeClass = scala.reflect.classTag[T].runtimeClass
+    val targetModuleSymbol = currentMirror.classSymbol(runtimeClass).companion
+    val parentClass = scala.reflect.classTag[SupportsNamedArguments].runtimeClass
+    val parentSymbol = currentMirror.classSymbol(parentClass)
+
+    if(targetModuleSymbol == scala.reflect.runtime.universe.NoSymbol) {
+      throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
+    }
+
+    val moduleClassSymbol = targetModuleSymbol.asModule.moduleClass.asClass
+    if (!moduleClassSymbol.baseClasses.contains(parentSymbol)) {
+      throw QueryCompilationErrors.namedArgumentsNotSupported(functionName)
+    }
+    if (currentMirror.runtimeClass(moduleClassSymbol).getEnclosingClass != null) {
+      throw QueryCompilationErrors.cannotObtainCompanionObjectInstance(functionName)
+    }
+    val instance = currentMirror.reflectModule(targetModuleSymbol.asModule)
+      .instance.asInstanceOf[SupportsNamedArguments]
+    if (instance.functionSignatures.size != 1) {
+      throw QueryCompilationErrors.multipleFunctionSignatures(
+        functionName, instance.functionSignatures)
+    }
+    instance.rearrange(instance.functionSignatures.head, expressions, functionName)
+  }
+
+  final def defaultRearrange(functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters: Seq[NamedArgument] = functionSignature.parameters
+    val firstNamedArgIdx: Int = args.indexWhere(_.isInstanceOf[NamedArgumentExpression])
+    val (positionalArgs, namedArgs) = args.splitAt(firstNamedArgIdx)
+    val namedParameters: Seq[NamedArgument] = parameters.drop(positionalArgs.size)
+
+    // Performing some checking to ensure valid argument list
+    val allParameterNames: Seq[String] = parameters.map(_.name)
+    val parameterNamesSet: Set[String] = allParameterNames.toSet
+    val assignedParameterSet = collection.mutable.Set[String](
+      allParameterNames.take(positionalArgs.size): _*)
+    for (arg <- namedArgs) {
+      arg match {
+        case namedArg: NamedArgumentExpression =>
+          if (assignedParameterSet.contains(namedArg.key)) {
+            throw QueryCompilationErrors.duplicateRoutineParameterAssignment(
+              functionName, namedArg.key)
+          }
+          if (!parameterNamesSet.contains(namedArg.key)) {
+            throw QueryCompilationErrors.unrecognizedParameterName(functionName, namedArg.key,
+              parameterNamesSet.toSeq)
+          }
+          assignedParameterSet.add(namedArg.key)
+        case _ =>
+          throw QueryCompilationErrors.unexpectedPositionalArgument(functionName)
+      }
+    }
+
+    // Construct a map from argument name to value for argument rearrangement
+    val namedArgMap = namedArgs.map { arg =>
+      val namedArg = arg.asInstanceOf[NamedArgumentExpression]
+      namedArg.key -> namedArg.value
+    }.toMap
+
+    // Rearrange named arguments to match their positional order
+    val rearrangedNamedArgs: Seq[Expression] = namedParameters.map { param =>
+      namedArgMap.getOrElse(
+        param.name,
+        if (param.default.isEmpty) {
+          throw QueryCompilationErrors.requiredParameterNotFound(functionName, param.name)
+        } else {
+          param.default.get
+        }
+      )
+    }
+    positionalArgs ++ rearrangedNamedArgs
+  }
+}
+
+/**
+ * Identifies which forms of provided argument values are expected for each call
+ * to the associated SQL function
+ */
+trait NamedArgumentType
+
+/**
+ * Represents a named argument that expects a scalar value of one specific DataType
+ *
+ * @param dataType The data type of some argument
+ */
+case class FixedArgumentType(dataType: AbstractDataType) extends NamedArgumentType
+
+/**
+ * Represents a parameter of a function expression. Function expressions should use this class
+ * to construct the argument lists returned in [[SupportsNamedArguments.functionSignatures]]
+ *
+ * @param name     The name of the string.
+ * @param dataType The datatype of the argument.
+ * @param default  The default value of the argument. If the default is none, then that means the
+ *                 argument is required. If no argument is provided, an exception is thrown.
+ */
+case class NamedArgument(
+    name: String,
+                          dataType: NamedArgumentType,

Review Comment:
   My bad. Will make sure to at the very least avoid this kind of glaring indentation problem.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1262224937


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -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" : {
+       "POSITIONAL_AND_NAMED_ARGUMENT_DOUBLE_REFERENCE" : {
+         "message" : [
+           "A positional argument and named argument both referred to the same parameter."
+         ]
+       },
+      "DOUBLE_NAMED_ARGUMENT_REFERENCE" : {

Review Comment:
   ```suggestion
         "MULTI_NAMED_ARGUMENT_REFERENCE" : {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] anchovYu commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "anchovYu (via GitHub)" <gi...@apache.org>.
anchovYu commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1258878085


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -1016,5 +1029,14 @@ object TableFunctionRegistry {
 }
 
 trait ExpressionBuilder {
+  def functionSignatures: Seq[FunctionSignature] = Nil

Review Comment:
   +1



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] anchovYu commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "anchovYu (via GitHub)" <gi...@apache.org>.
anchovYu commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1258916558


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala:
##########
@@ -344,4 +346,13 @@ object Mask extends SupportsNamedArguments {
       strArg, upperCharArg, lowerCharArg, digitCharArg, otherCharArg))
     Seq(functionSignature)
   }
+
+  override def build(funcName: String, expressions: Seq[Expression]): Expression = {
+    if (expressions.length < 1 || expressions.length > 5) {
+      throw QueryCompilationErrors.wrongNumArgsError(
+        funcName, Seq(1, 2, 3, 4, 5), expressions.length)
+    }
+    new Mask(expressions(0), expressions(1), expressions(2), expressions(3), expressions(4))

Review Comment:
   I think clearly define our plan on new functions that supports named arguments - what is our behavior when users use all positional arguments? New functions may only have one constructor without overloarding but with well defined function signatures, e.g. Seq(`a` required, `b` optional). If users only pass func(1), do we throw exception that it does not match the argument count, or do our framework just consider this 1 as `a`, and fill the `b` with default value?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1262773644


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -50,6 +50,90 @@ import org.apache.spark.sql.types._
  */
 private[sql] object QueryCompilationErrors extends QueryErrorsBase {
 
+  def unexpectedRequiredParameterInFunctionSignature(
+      functionName: String, functionSignature: FunctionSignature) : Throwable = {
+    val errorMessage = s"Function $functionName has an unexpected required argument for" +
+      s" the provided function signature $functionSignature. All required arguments should" +
+      s" come before optional arguments."
+    SparkException.internalError(errorMessage)
+  }
+
+  def multipleFunctionSignatures(functionName: String,
+      functionSignatures: Seq[FunctionSignature]): Throwable = {
+    var errorMessage = s"Function $functionName cannot have multiple method signatures." +
+      s" The function signatures found were: \n"
+    for (functionSignature <- functionSignatures) {
+      errorMessage = errorMessage + s"${functionSignature}\n"
+    }
+    SparkException.internalError(errorMessage)
+  }
+
+  def namedArgumentsNotSupported(functionName: String) : Throwable = {
+    new AnalysisException(
+      errorClass = "NAMED_ARGUMENTS_NOT_SUPPORTED",
+      messageParameters = Map("functionName" -> toSQLId(functionName))
+    )
+  }
+
+  def positionalAndNamedArgumentDoubleReference(
+      functionName: String, parameterName: String) : Throwable = {
+    val errorClass =
+      "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.POSITIONAL_AND_NAMED_ARGUMENT_DOUBLE_REFERENCE"
+    new AnalysisException(
+      errorClass = errorClass,
+      messageParameters = Map(
+        "functionName" -> toSQLId(functionName),
+        "parameterName" -> toSQLId(parameterName))
+    )
+  }
+
+  def doubleNamedArgumentReference(
+      functionName: String, parameterName: String): Throwable = {
+    val errorClass =
+      "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.DOUBLE_NAMED_ARGUMENT_REFERENCE"
+    new AnalysisException(
+      errorClass = errorClass,
+      messageParameters = Map(
+        "functionName" -> toSQLId(functionName),
+        "parameterName" -> toSQLId(parameterName))
+    )
+  }
+
+  def requiredParameterNotFound(
+      functionName: String, parameterName: String) : Throwable = {
+    new AnalysisException(
+      errorClass = "REQUIRED_PARAMETER_NOT_FOUND",
+      messageParameters = Map(
+        "functionName" -> toSQLId(functionName),
+        "parameterName" -> toSQLId(parameterName))
+    )
+  }
+
+  def unrecognizedParameterName(
+      functionName: String, argumentName: String, candidates: Seq[String]): Throwable = {
+    import org.apache.spark.sql.catalyst.util.StringUtils.orderSuggestedIdentifiersBySimilarity
+
+    val inputs = candidates.map(candidate => Seq(candidate)).toSeq
+    val recommendations = orderSuggestedIdentifiersBySimilarity(argumentName, inputs)
+      .take(3)
+    var candidatesString = ""
+    recommendations.foreach(candidatesString += _ + " ")

Review Comment:
   Could you just use `mkString`, please.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1263018019


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/FunctionBuilderBase.scala:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+trait FunctionBuilderBase[T] {
+  /**
+   * A method that returns the signatures of overloads that are associated with this function.
+   * Each function signature includes a list of parameters to which the analyzer can
+   * compare a function call with provided arguments to determine if that function
+   * call is a match for the function signature.
+   *
+   * @return a list of function signatures
+   */
+  def functionSignatures: Option[Seq[FunctionSignature]] = None
+
+  /**
+   * This function rearranges the arguments provided during function invocation in positional order
+   * according to the function signature. This method will fill in the default values if optional
+   * parameters do not have their values specified. Any function which supports named arguments
+   * will have this routine invoked, even if no named arguments are present in the argument list.
+   * This is done to eliminate constructor overloads in some methods which use them for default
+   * values prior to the implementation of the named argument framework. This function will also
+   * check if the number of arguments are correct. If that is not the case, then an error will be
+   * thrown.
+   *
+   * IMPORTANT: This method will be called before the [[FunctionBuilderBase.build]] method is
+   * invoked. It is guaranteed that the expressions provided to the [[FunctionBuilderBase.build]]
+   * functions forms a valid set of argument expressions that can be used in the construction of
+   * the function expression.
+   *
+   * @param expectedSignature The method signature which we rearrange our arguments according to
+   * @param providedArguments The list of arguments passed from function invocation
+   * @param functionName The name of the function
+   * @return The rearranged argument list with arguments in positional order
+   */
+  def rearrange(
+                 expectedSignature: FunctionSignature,
+                 providedArguments: Seq[Expression],
+                 functionName: String) : Seq[Expression] = {
+    NamedArgumentsSupport.defaultRearrange(expectedSignature, providedArguments, functionName)
+  }
+
+  def build(funcName: String, expressions: Seq[Expression]): T
+}
+
+object NamedArgumentsSupport {
+  /**
+   * This method is the default routine which rearranges the arguments in positional order according
+   * to the function signature provided. This will also fill in any default values that exists for
+   * optional arguments. This method will also be invoked even if there are no named arguments in
+   * the argument list.
+   *
+   * @param functionSignature The function signature that defines the positional ordering
+   * @param args The argument list provided in function invocation
+   * @param functionName The name of the function
+   * @return A list of arguments rearranged in positional order defined by the provided signature
+   */
+  final def defaultRearrange(
+      functionSignature: FunctionSignature,
+      args: Seq[Expression],
+      functionName: String): Seq[Expression] = {
+    val parameters: Seq[NamedArgument] = functionSignature.parameters
+    if (parameters.dropWhile(_.default.isEmpty).exists(_.default.isEmpty)) {
+      throw QueryCompilationErrors.unexpectedRequiredParameterInFunctionSignature(
+        functionName, functionSignature)
+    }
+
+    val (positionalArgs, namedArgs) = args.span(!_.isInstanceOf[NamedArgumentExpression])
+    val namedParameters: Seq[NamedArgument] = parameters.drop(positionalArgs.size)
+
+    // The following loop checks for the following:
+    // 1. Unrecognized parameter names
+    // 2. Duplicate routine parameter assignments
+    val allParameterNames: Seq[String] = parameters.map(_.name)
+    val parameterNamesSet: Set[String] = allParameterNames.toSet
+    val positionalParametersSet = allParameterNames.take(positionalArgs.size).toSet
+    val namedParametersSet = collection.mutable.Set[String]()
+
+    for (arg <- namedArgs) {
+      arg match {
+        case namedArg: NamedArgumentExpression =>
+          val parameterName = namedArg.key
+          if (!parameterNamesSet.contains(parameterName)) {
+            throw QueryCompilationErrors.unrecognizedParameterName(functionName, namedArg.key,
+              parameterNamesSet.toSeq)
+          }
+          if (positionalParametersSet.contains(parameterName)) {
+            throw QueryCompilationErrors.positionalAndNamedArgumentDoubleReference(
+              functionName, namedArg.key)
+          }
+          if (namedParametersSet.contains(parameterName)) {
+            throw QueryCompilationErrors.doubleNamedArgumentReference(
+              functionName, namedArg.key)
+          }
+          namedParametersSet.add(namedArg.key)
+        case _ =>
+          throw QueryCompilationErrors.unexpectedPositionalArgument(functionName)
+      }
+    }
+
+    // Check argument list size against provided parameter list length
+    if (parameters.size < args.length) {

Review Comment:
   I think we already have tests for these like in the context of explode for example.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] learningchess2003 commented on a diff in pull request #41864: [SPARK-44059] Add analyzer support of named arguments for built-in functions

Posted by "learningchess2003 (via GitHub)" <gi...@apache.org>.
learningchess2003 commented on code in PR #41864:
URL: https://github.com/apache/spark/pull/41864#discussion_r1261779139


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala:
##########
@@ -431,6 +432,22 @@ case class Explode(child: Expression) extends ExplodeBase {
     copy(child = newChild)
 }
 
+trait ExplodeGeneratorBuilderBase extends GeneratorBuilder {
+  override def functionSignatures: Option[Seq[FunctionSignature]] =
+    Some(Seq(FunctionSignature(Seq(NamedArgument("collection")))))

Review Comment:
   To best explain this:
   
   1. The option is used to check if named arguments will be supported for this expression builder.
   2. The seq is potentially useful in the future for function overloads.
   3. There can be additional information that can be packaged into the FunctionSignature in the future.
   4. NamedArguments can hold additional information like default values. 
   
   So most of these things are pretty useful. For now, I think it makes sense to keep as is.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org