You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/04/07 23:26:14 UTC

[GitHub] [spark] sunchao opened a new pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

sunchao opened a new pull request #32082:
URL: https://github.com/apache/spark/pull/32082


   Co-Authored-By: Chao Sun <su...@apple.com>
   Co-Authored-By: Ryan Blue <rb...@netflix.com>
   
   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   This implements function resolution and evaluation for functions registered through V2 FunctionCatalog [SPARK-27658](https://issues.apache.org/jira/browse/SPARK-27658). In particular:
   - Added documentation for how to define the "magic method" in `ScalarFunction`.
   - Added a new expression `ApplyFunctionExpression` which evaluates input by delegating to `ScalarFunction.produceResult` method.
   - added a new expression `V2Aggregator` which is a type of `TypedImperativeAggregate`. It's a wrapper of V2 `AggregateFunction` and mostly delegate methods to the implementation of the latter. It also uses plain Java serde for intermediate state.
   - Added function resolution logic for `ScalarFunction` and `AggregateFunction` in `Analyzer`. 
     + For `ScalarFunction` this checks if the magic method is implemented through Java reflection, and create a `Invoke` expression if so. Otherwise, it checks if the default `produceResult` is overridden. If so, it creates a `ApplyFunctionExpression` which evaluates through `InternalRow`. Otherwise an analysis exception is thrown.
    + For `AggregateFunction`, this checks if the `update` method is overridden. If so, it converts it to `V2Aggregator`. Otherwise an analysis exception is thrown similar to the case of `ScalarFunction`.
   - Extended existing `InMemoryTableCatalog` to add the function catalog capability. Also renamed it to `InMemoryCatalog` since it no longer only covers tables.
   
   **Note**: this currently can successfully detect whether a subclass overrides the default `produceResult` or `update` method from the parent interface **only for Java implementations**. It seems in Scala it's hard to differentiate whether a subclass overrides a default method from its parent interface. In this case, it will be a runtime error instead of analysis error.
   
   Also this doesn't cover `V2SessionCatalog`. Ideally we want to extend that to support function catalog as well but the APIs are different between V2 FunctionCatalog and those in V1 `SessionCatalog`.
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   As V2 FunctionCatalog APIs are finalized, we should integrate it with function resolution and evaluation process so that they are actually useful.
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   Yes, now a function exposed through V2 FunctionCatalog can be analyzed and evaluated.
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   
   Added new unit 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.

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815567034


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/41642/
   


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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r614231047



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java
##########
@@ -30,10 +30,71 @@
  * <p>
  * The JVM type of result values produced by this function must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.
+ * <p>
+ * <b>IMPORTANT</b>: the default implementation of {@link #produceResult} throws
+ * {@link UnsupportedOperationException}. Users can choose to override this method, or implement
+ * a "magic method" with name {@link #MAGIC_METHOD_NAME} which takes individual parameters

Review comment:
       Sure, will update it. It currently says "If no direct method is found or when not using codegen, Spark will call produceResult" which is different from what this PR does, which always call the magic method if it's defined. Do you think it matters much?




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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-818001611


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/41807/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825177213


   **[Test build #137820 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137820/testReport)** for PR 32082 at commit [`66dac00`](https://github.com/apache/spark/commit/66dac00f7d417452dc7eb268475b22ddcbb40075).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public 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.

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r621207960



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/AggregateFunction.java
##########
@@ -25,12 +25,9 @@
 /**
  * Interface for a function that produces a result value by aggregating over multiple input rows.
  * <p>
- * For each input row, Spark will call an update method that corresponds to the
- * {@link #inputTypes() input data types}. The expected JVM argument types must be the types used by
- * Spark's InternalRow API. If no direct method is found or when not using codegen, Spark will call
- * update with {@link InternalRow}.
- * <p>
- * The JVM type of result values produced by this function must be the type used by Spark's
+ * For each input row, Spark will call the {@link #update} method which should evaluate the row

Review comment:
       `which should evaluate the row`
   
   This is a bit confusing. How about
   ```
   Spark will call the {@link #update} method to update the aggregation state with the input row.
   ```




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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825444942


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42378/
   


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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r614083003



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/V2Aggregator.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.connector.catalog.functions.{AggregateFunction => V2AggregateFunction}
+import org.apache.spark.sql.types.DataType
+
+case class V2Aggregator[BUF <: java.io.Serializable, OUT](
+    aggrFunc: V2AggregateFunction[BUF, OUT],
+    children: Seq[Expression],
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0) extends TypedImperativeAggregate[BUF] {
+  override def createAggregationBuffer(): BUF = aggrFunc.newAggregationState()
+
+  override def update(buffer: BUF, input: InternalRow): BUF = aggrFunc.update(buffer, input)

Review comment:
       Similar to `ScalarFunction`, we can document that `invokeUpdate` is the magic method 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.

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r621972605



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java
##########
@@ -23,17 +23,67 @@
 /**
  * Interface for a function that produces a result value for each input row.
  * <p>
- * For each input row, Spark will call a produceResult method that corresponds to the
- * {@link #inputTypes() input data types}. The expected JVM argument types must be the types used by
- * Spark's InternalRow API. If no direct method is found or when not using codegen, Spark will call
- * {@link #produceResult(InternalRow)}.
+ * To evaluate each input row, Spark will first try to lookup and use a "magic method" (described
+ * below) through Java reflection. If the method is not found, Spark will call
+ * {@link #produceResult(InternalRow)} as a fallback approach.
  * <p>
  * The JVM type of result values produced by this function must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.
+ * <p>
+ * <b>IMPORTANT</b>: the default implementation of {@link #produceResult} throws
+ * {@link UnsupportedOperationException}. Users can choose to override this method, or implement
+ * a "magic method" with name {@link #MAGIC_METHOD_NAME} which takes individual parameters
+ * instead of a {@link InternalRow}. The magic method will be loaded by Spark through Java
+ * reflection and will also provide better performance in general, due to optimizations such as
+ * codegen, removal of Java boxing, etc.
+ *
+ * For example, a scalar UDF for adding two integers can be defined as follow with the magic
+ * method approach:
+ *
+ * <pre>
+ *   public class IntegerAdd implements{@code ScalarFunction<Integer>} {
+ *     public int invoke(int left, int right) {

Review comment:
       @sunchao can you spend some time on the API design? I'd love to see this 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.

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825187750






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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r620440838



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2016,88 +2029,210 @@ class Analyzer(override val catalogManager: CatalogManager)
                   name, other.getClass.getCanonicalName)
               }
             }
-          case u @ UnresolvedFunction(funcId, arguments, isDistinct, filter, ignoreNulls) =>
+
+          case u @ UnresolvedFunction(AsFunctionIdentifier(ident), arguments,

Review comment:
       ditto, it clearer to use if else here
   ```
   case u @ UnresolvedFunction(nameParts, ...) =>
     val CatalogAndIdentifier(catalog, v2Ident) = nameParts
     def lookupPersistentFunction(): Expression =  if (CatalogV2Util.isSessionCatalog(catalog)) {
       processFunctionExpr(v1SessionCatalog.lookupFunction ...
     } else {
       v2 code path
     }
   
     if (nameParts.length == 1) {
       v1SessionCatalog.lookupBuiltinOrTempFunction....map(...).getOrElse(lookupPersistentFunction)
     } else {
       lookupPersistentFunction
     }
   ```




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827382359


   **[Test build #137992 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137992/testReport)** for PR 32082 at commit [`c453b64`](https://github.com/apache/spark/commit/c453b64ae669c2be28a9467be8b5111a7563f2ab).


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-821554886


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42070/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815562644


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/41642/
   


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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r618171718



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
##########
@@ -269,23 +269,24 @@ case class UnresolvedGenerator(name: FunctionIdentifier, children: Seq[Expressio
 }
 
 case class UnresolvedFunction(
-    name: FunctionIdentifier,
+    multipartIdentifier: Seq[String],
     arguments: Seq[Expression],
     isDistinct: Boolean,
     filter: Option[Expression] = None,
     ignoreNulls: Boolean = false)
   extends Expression with Unevaluable {
+  import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
 
   override def children: Seq[Expression] = arguments ++ filter.toSeq
 
   override def dataType: DataType = throw new UnresolvedException("dataType")
   override def nullable: Boolean = throw new UnresolvedException("nullable")
   override lazy val resolved = false
 
-  override def prettyName: String = name.unquotedString
+  override def prettyName: String = multipartIdentifier.quoted
   override def toString: String = {
     val distinct = if (isDistinct) "distinct " else ""
-    s"'$name($distinct${children.mkString(", ")})"
+    s"'${multipartIdentifier.quoted}($distinct${children.mkString(", ")})"

Review comment:
       I think it indicates unresolved




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

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] rdblue commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617886188



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2021,88 +2032,218 @@ class Analyzer(override val catalogManager: CatalogManager)
                   name, other.getClass.getCanonicalName)
               }
             }
-          case u @ UnresolvedFunction(funcId, arguments, isDistinct, filter, ignoreNulls) =>
+
+          case u @ UnresolvedFunction(AsFunctionIdentifier(ident), arguments,
+          isDistinct, filter, ignoreNulls) =>
             withPosition(u) {
-              v1SessionCatalog.lookupFunction(funcId, arguments) match {
-                // AggregateWindowFunctions are AggregateFunctions that can only be evaluated within
-                // the context of a Window clause. They do not need to be wrapped in an
-                // AggregateExpression.
-                case wf: AggregateWindowFunction =>
-                  if (isDistinct) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      wf.prettyName, "DISTINCT")
-                  } else if (filter.isDefined) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      wf.prettyName, "FILTER clause")
-                  } else if (ignoreNulls) {
-                    wf match {
-                      case nthValue: NthValue =>
-                        nthValue.copy(ignoreNulls = ignoreNulls)
-                      case _ =>
-                        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                          wf.prettyName, "IGNORE NULLS")
-                    }
-                  } else {
-                    wf
-                  }
-                case owf: FrameLessOffsetWindowFunction =>
-                  if (isDistinct) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      owf.prettyName, "DISTINCT")
-                  } else if (filter.isDefined) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      owf.prettyName, "FILTER clause")
-                  } else if (ignoreNulls) {
-                    owf match {
-                      case lead: Lead =>
-                        lead.copy(ignoreNulls = ignoreNulls)
-                      case lag: Lag =>
-                        lag.copy(ignoreNulls = ignoreNulls)
-                    }
-                  } else {
-                    owf
-                  }
-                // We get an aggregate function, we need to wrap it in an AggregateExpression.
-                case agg: AggregateFunction =>
-                  if (filter.isDefined && !filter.get.deterministic) {
-                    throw QueryCompilationErrors.nonDeterministicFilterInAggregateError
-                  }
-                  if (ignoreNulls) {
-                    val aggFunc = agg match {
-                      case first: First => first.copy(ignoreNulls = ignoreNulls)
-                      case last: Last => last.copy(ignoreNulls = ignoreNulls)
-                      case _ =>
-                        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                          agg.prettyName, "IGNORE NULLS")
-                    }
-                    AggregateExpression(aggFunc, Complete, isDistinct, filter)
-                  } else {
-                    AggregateExpression(agg, Complete, isDistinct, filter)
-                  }
-                // This function is not an aggregate function, just return the resolved one.
-                case other if isDistinct =>
-                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                    other.prettyName, "DISTINCT")
-                case other if filter.isDefined =>
-                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                    other.prettyName, "FILTER clause")
-                case other if ignoreNulls =>
-                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                    other.prettyName, "IGNORE NULLS")
-                case e: String2TrimExpression if arguments.size == 2 =>
-                  if (trimWarningEnabled.get) {
-                    log.warn("Two-parameter TRIM/LTRIM/RTRIM function signatures are deprecated." +
-                      " Use SQL syntax `TRIM((BOTH | LEADING | TRAILING)? trimStr FROM str)`" +
-                      " instead.")
-                    trimWarningEnabled.set(false)
-                  }
-                  e
-                case other =>
-                  other
+              processFunctionExpr(v1SessionCatalog.lookupFunction(ident, arguments),
+                arguments, isDistinct, filter, ignoreNulls)
+            }
+
+          case u @ UnresolvedFunction(parts, arguments, isDistinct, filter, ignoreNulls) =>
+            withPosition(u) {
+              // resolve built-in or temporary functions with v2 catalog
+              val resultExpression = if (parts.length == 1) {
+                v1SessionCatalog.lookupBuiltinOrTempFunction(parts.head, arguments).map(
+                  processFunctionExpr(_, arguments, isDistinct, filter, ignoreNulls)
+                )
+              } else {
+                None
               }
+
+              resultExpression.getOrElse(
+                expandIdentifier(parts) match {
+                  case NonSessionCatalogAndIdentifier(catalog: FunctionCatalog, ident) =>
+                    lookupV2Function(catalog, ident, arguments, isDistinct, filter, ignoreNulls)
+                  case _ => u
+                }
+              )
             }
         }
     }
+
+    /**
+     * Check if the input `fn` implements the given `methodName`. If `inputType` is set, it also
+     * tries to match it against the declared parameter types.
+     */
+    private def findMethod(
+        fn: BoundFunction,
+        methodName: String,
+        inputTypeOpt: Option[Seq[DataType]] = None): Option[Method] = {
+      val cls = fn.getClass
+      inputTypeOpt match {
+        case Some(inputType) =>
+          try {
+            val argClasses = inputType.map(ScalaReflection.dataTypeJavaClass)
+            Some(cls.getDeclaredMethod(methodName, argClasses: _*))
+          } catch {
+            case _: NoSuchMethodException =>
+              None
+          }
+        case None =>
+          cls.getDeclaredMethods.find(_.getName == methodName)
+      }
+    }
+
+    private def processFunctionExpr(
+        expr: Expression,
+        arguments: Seq[Expression],
+        isDistinct: Boolean,
+        filter: Option[Expression],
+        ignoreNulls: Boolean): Expression = expr match {
+      // AggregateWindowFunctions are AggregateFunctions that can only be evaluated within
+      // the context of a Window clause. They do not need to be wrapped in an
+      // AggregateExpression.
+      case wf: AggregateWindowFunction =>
+        if (isDistinct) {
+          throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+            wf.prettyName, "DISTINCT")
+        } else if (filter.isDefined) {
+          throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+            wf.prettyName, "FILTER clause")
+        } else if (ignoreNulls) {
+          wf match {
+            case nthValue: NthValue =>
+              nthValue.copy(ignoreNulls = ignoreNulls)
+            case _ =>
+              throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+                wf.prettyName, "IGNORE NULLS")
+          }
+        } else {
+          wf
+        }
+      case owf: FrameLessOffsetWindowFunction =>
+        if (isDistinct) {
+          throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+            owf.prettyName, "DISTINCT")
+        } else if (filter.isDefined) {
+          throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+            owf.prettyName, "FILTER clause")
+        } else if (ignoreNulls) {
+          owf match {
+            case lead: Lead =>
+              lead.copy(ignoreNulls = ignoreNulls)
+            case lag: Lag =>
+              lag.copy(ignoreNulls = ignoreNulls)
+          }
+        } else {
+          owf
+        }
+      // We get an aggregate function, we need to wrap it in an AggregateExpression.
+      case agg: AggregateFunction =>
+        if (filter.isDefined && !filter.get.deterministic) {
+          throw QueryCompilationErrors.nonDeterministicFilterInAggregateError
+        }
+        if (ignoreNulls) {
+          val aggFunc = agg match {
+            case first: First => first.copy(ignoreNulls = ignoreNulls)
+            case last: Last => last.copy(ignoreNulls = ignoreNulls)
+            case _ =>
+              throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+                agg.prettyName, "IGNORE NULLS")
+          }
+          AggregateExpression(aggFunc, Complete, isDistinct, filter)
+        } else {
+          AggregateExpression(agg, Complete, isDistinct, filter)
+        }
+      // This function is not an aggregate function, just return the resolved one.
+      case other if isDistinct =>
+        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+          other.prettyName, "DISTINCT")
+      case other if filter.isDefined =>
+        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+          other.prettyName, "FILTER clause")
+      case other if ignoreNulls =>
+        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+          other.prettyName, "IGNORE NULLS")
+      case e: String2TrimExpression if arguments.size == 2 =>
+        if (trimWarningEnabled.get) {
+          log.warn("Two-parameter TRIM/LTRIM/RTRIM function signatures are deprecated." +
+            " Use SQL syntax `TRIM((BOTH | LEADING | TRAILING)? trimStr FROM str)`" +
+            " instead.")
+          trimWarningEnabled.set(false)
+        }
+        e
+      case other =>
+        other
+    }
+
+    private def lookupV2Function(
+        catalog: FunctionCatalog,
+        ident: Identifier,
+        arguments: Seq[Expression],
+        isDistinct: Boolean,
+        filter: Option[Expression],
+        ignoreNulls: Boolean): Expression = {
+      val unbound = catalog.loadFunction(ident)
+      val inputType = StructType(arguments.zipWithIndex.map {
+        case (exp, pos) => StructField(s"_$pos", exp.dataType, exp.nullable)
+      })
+      val bound = try {
+        unbound.bind(inputType)
+      } catch {
+        case unsupported: UnsupportedOperationException =>
+          failAnalysis(s"Function '${unbound.name}' cannot process input: " +
+            s"(${arguments.map(_.dataType.simpleString).mkString(", ")}): " +
+            unsupported.getMessage)
+      }
+
+      bound match {
+        case scalarFunc: ScalarFunction[_] =>
+          if (isDistinct) {
+            throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+              scalarFunc.name(), "DISTINCT")
+          } else if (filter.isDefined) {
+            throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+              scalarFunc.name(), "FILTER clause")
+          } else if (ignoreNulls) {
+            throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+              scalarFunc.name(), "IGNORE NULLS")
+          } else {
+            // TODO: implement type coercion by looking at input type from the UDF. We may
+            //  also want to check if the parameter types from the magic method match the
+            //  input type through `BoundFunction.inputTypes`.
+            val argClasses = inputType.fields.map(_.dataType)
+            findMethod(scalarFunc, MAGIC_METHOD_NAME, Some(argClasses)) match {
+              case Some(_) =>
+                val caller = Literal.create(scalarFunc, ObjectType(scalarFunc.getClass))
+                Invoke(caller, MAGIC_METHOD_NAME, scalarFunc.resultType(),
+                  arguments, returnNullable = scalarFunc.isResultNullable)
+              case _ =>
+                // TODO: handle functions defined in Scala too - in Scala, even if a
+                //  subclass do not override the default method in parent interface
+                //  defined in Java, the method can still be found from
+                //  `getDeclaredMethod`.
+                findMethod(scalarFunc, "produceResult", Some(Seq(inputType))) match {

Review comment:
       Isn't this supposed to look up the method that accepts `InternalRow`? But it doesn't look like that's what happens here because the sequence of input types is passed.




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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r615600415



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1967,6 +1969,9 @@ class Analyzer(override val catalogManager: CatalogManager)
     override def apply(plan: LogicalPlan): LogicalPlan = {
       val externalFunctionNameSet = new mutable.HashSet[FunctionIdentifier]()
       plan.resolveExpressions {
+        case f @ UnresolvedFunction(NonSessionCatalogAndIdentifier(_, _), _, _, _, _) =>
+          // no-op if this is from a v2 catalog

Review comment:
       SGTM




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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827426212


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137992/
   


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

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] SparkQA removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827422624


   **[Test build #137994 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137994/testReport)** for PR 32082 at commit [`790d27f`](https://github.com/apache/spark/commit/790d27f900c0ed456c198a5175410961223990f8).


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

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] rdblue commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617881687



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1963,6 +1966,14 @@ class Analyzer(override val catalogManager: CatalogManager)
     override def apply(plan: LogicalPlan): LogicalPlan = {
       val externalFunctionNameSet = new mutable.HashSet[FunctionIdentifier]()
       plan.resolveExpressions {
+        case f @ UnresolvedFunction(NonSessionCatalogAndIdentifier(catalog, name), _, _, _, _) =>

Review comment:
       Why is this done here and not in analysis checks?




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827350141


   **[Test build #137987 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137987/testReport)** for PR 32082 at commit [`f25b5e6`](https://github.com/apache/spark/commit/f25b5e683f796c353296adb519b4882e37fbc213).


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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r615227429



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2053,9 +2063,74 @@ class Analyzer(override val catalogManager: CatalogManager)
                   name, other.getClass.getCanonicalName)
               }
             }
-          case u @ UnresolvedFunction(funcId, arguments, isDistinct, filter, ignoreNulls) =>
+          case UnresolvedFunction(NonSessionCatalogAndIdentifier(v2Catalog, ident), arguments,

Review comment:
       This logic is not correct - when there is no namespace part in the identifier, it should check built-in and temp functions first, and also handle function references in views. Only if this part failed it should use the default catalog (whether that be the V1 session catalog or a V2 catalog). Will fix 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.

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] rdblue commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617882693



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2021,88 +2032,218 @@ class Analyzer(override val catalogManager: CatalogManager)
                   name, other.getClass.getCanonicalName)
               }
             }
-          case u @ UnresolvedFunction(funcId, arguments, isDistinct, filter, ignoreNulls) =>
+
+          case u @ UnresolvedFunction(AsFunctionIdentifier(ident), arguments,
+          isDistinct, filter, ignoreNulls) =>
             withPosition(u) {
-              v1SessionCatalog.lookupFunction(funcId, arguments) match {
-                // AggregateWindowFunctions are AggregateFunctions that can only be evaluated within
-                // the context of a Window clause. They do not need to be wrapped in an
-                // AggregateExpression.
-                case wf: AggregateWindowFunction =>
-                  if (isDistinct) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      wf.prettyName, "DISTINCT")
-                  } else if (filter.isDefined) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      wf.prettyName, "FILTER clause")
-                  } else if (ignoreNulls) {
-                    wf match {
-                      case nthValue: NthValue =>
-                        nthValue.copy(ignoreNulls = ignoreNulls)
-                      case _ =>
-                        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                          wf.prettyName, "IGNORE NULLS")
-                    }
-                  } else {
-                    wf
-                  }
-                case owf: FrameLessOffsetWindowFunction =>
-                  if (isDistinct) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      owf.prettyName, "DISTINCT")
-                  } else if (filter.isDefined) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      owf.prettyName, "FILTER clause")
-                  } else if (ignoreNulls) {
-                    owf match {
-                      case lead: Lead =>
-                        lead.copy(ignoreNulls = ignoreNulls)
-                      case lag: Lag =>
-                        lag.copy(ignoreNulls = ignoreNulls)
-                    }
-                  } else {
-                    owf
-                  }
-                // We get an aggregate function, we need to wrap it in an AggregateExpression.
-                case agg: AggregateFunction =>
-                  if (filter.isDefined && !filter.get.deterministic) {
-                    throw QueryCompilationErrors.nonDeterministicFilterInAggregateError
-                  }
-                  if (ignoreNulls) {
-                    val aggFunc = agg match {
-                      case first: First => first.copy(ignoreNulls = ignoreNulls)
-                      case last: Last => last.copy(ignoreNulls = ignoreNulls)
-                      case _ =>
-                        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                          agg.prettyName, "IGNORE NULLS")
-                    }
-                    AggregateExpression(aggFunc, Complete, isDistinct, filter)
-                  } else {
-                    AggregateExpression(agg, Complete, isDistinct, filter)
-                  }
-                // This function is not an aggregate function, just return the resolved one.
-                case other if isDistinct =>
-                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                    other.prettyName, "DISTINCT")
-                case other if filter.isDefined =>
-                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                    other.prettyName, "FILTER clause")
-                case other if ignoreNulls =>
-                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                    other.prettyName, "IGNORE NULLS")
-                case e: String2TrimExpression if arguments.size == 2 =>
-                  if (trimWarningEnabled.get) {
-                    log.warn("Two-parameter TRIM/LTRIM/RTRIM function signatures are deprecated." +
-                      " Use SQL syntax `TRIM((BOTH | LEADING | TRAILING)? trimStr FROM str)`" +
-                      " instead.")
-                    trimWarningEnabled.set(false)
-                  }
-                  e
-                case other =>
-                  other
+              processFunctionExpr(v1SessionCatalog.lookupFunction(ident, arguments),
+                arguments, isDistinct, filter, ignoreNulls)
+            }
+
+          case u @ UnresolvedFunction(parts, arguments, isDistinct, filter, ignoreNulls) =>
+            withPosition(u) {
+              // resolve built-in or temporary functions with v2 catalog
+              val resultExpression = if (parts.length == 1) {

Review comment:
       Could this be a separate case instead?




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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r637708233



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ApplyFunctionExpression.scala
##########
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction
+import org.apache.spark.sql.types.DataType
+
+case class ApplyFunctionExpression(
+    function: ScalarFunction[_],
+    children: Seq[Expression]) extends Expression with UserDefinedExpression with CodegenFallback {
+  override def nullable: Boolean = function.isResultNullable
+  override def name: String = function.name()
+  override def dataType: DataType = function.resultType()
+
+  private lazy val reusedRow = new GenericInternalRow(children.size)

Review comment:
       sorry for the delay - just opened #32647 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.

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r621219144



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2095,9 +2104,101 @@ class Analyzer(override val catalogManager: CatalogManager)
                 case other =>
                   other
               }
+          }
+
+          case u @ UnresolvedFunction(nameParts, arguments, isDistinct, filter, ignoreNulls) =>
+            withPosition(u) {
+              expandIdentifier(nameParts) match {
+                case NonSessionCatalogAndIdentifier(catalog, ident) =>
+                  if (!catalog.isFunctionCatalog) {
+                    throw new AnalysisException(s"Trying to lookup function '$ident' in " +
+                      s"catalog '${catalog.name()}', but it is not a FunctionCatalog.")
+                  }
+
+                  val unbound = catalog.asFunctionCatalog.loadFunction(ident)
+                  val inputType = StructType(arguments.zipWithIndex.map {
+                    case (exp, pos) => StructField(s"_$pos", exp.dataType, exp.nullable)
+                  })
+                  val bound = try {
+                    unbound.bind(inputType)
+                  } catch {
+                    case unsupported: UnsupportedOperationException =>
+                      throw new AnalysisException(s"Function '${unbound.name}' cannot process " +
+                        s"input: (${arguments.map(_.dataType.simpleString).mkString(", ")}): " +
+                        unsupported.getMessage, cause = Some(unsupported))
+                  }
+
+                  bound match {
+                    case scalarFunc: ScalarFunction[_] =>
+                      if (isDistinct) {
+                        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+                          scalarFunc.name(), "DISTINCT")
+                      } else if (filter.isDefined) {
+                        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+                          scalarFunc.name(), "FILTER clause")
+                      } else if (ignoreNulls) {
+                        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+                          scalarFunc.name(), "IGNORE NULLS")
+                      } else {
+                        // TODO: implement type coercion by looking at input type from the UDF. We
+                        //  may also want to check if the parameter types from the magic method
+                        //  match the input type through `BoundFunction.inputTypes`.
+                        val argClasses = inputType.fields.map(_.dataType)
+                        findMethod(scalarFunc, MAGIC_METHOD_NAME, argClasses) match {
+                          case Some(_) =>
+                            val caller = Literal.create(scalarFunc, ObjectType(scalarFunc.getClass))
+                            Invoke(caller, MAGIC_METHOD_NAME, scalarFunc.resultType(),
+                              arguments, returnNullable = scalarFunc.isResultNullable)
+                          case _ =>
+                            // TODO: handle functions defined in Scala too - in Scala, even if a
+                            //  subclass do not override the default method in parent interface
+                            //  defined in Java, the method can still be found from
+                            //  `getDeclaredMethod`.
+                            // since `inputType` is a `StructType`, it is mapped to a `InternalRow`
+                            // which we can use to lookup the `produceResult` method.
+                            findMethod(scalarFunc, "produceResult", Seq(inputType)) match {
+                              case Some(_) =>
+                                ApplyFunctionExpression(scalarFunc, arguments)
+                              case None =>
+                                failAnalysis(s"ScalarFunction '${bound.name()}' neither implement" +
+                                  s" magic method nor override 'produceResult'")
+                            }
+                        }
+                      }
+                    case aggFunc: V2AggregateFunction[_, _] =>

Review comment:
       ditto, put into a new 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.

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815352262


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137036/
   


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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815413210


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137039/
   


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

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] SparkQA removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827355087


   **[Test build #137989 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137989/testReport)** for PR 32082 at commit [`97c29c3`](https://github.com/apache/spark/commit/97c29c3aef6dbcd8fd43d4f96bcd0238b932fc02).


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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827384320


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42509/
   


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

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] rdblue commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617876781



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala
##########
@@ -29,7 +29,7 @@ import org.scalatest.matchers.must.Matchers
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFormat, CatalogTable, CatalogTableType, ExternalCatalog, InMemoryCatalog, SessionCatalog}
 import org.apache.spark.sql.catalyst.dsl.plans._
-import org.apache.spark.sql.connector.{InMemoryTable, InMemoryTableCatalog}
+import org.apache.spark.sql.connector.{InMemoryTable, V2InMemoryCatalog}

Review comment:
       Can we avoid changing the `InMemoryTableCatalog` to keep changes smaller? It seems like we could create a separate `InMemoryFunctionCatalog` instead.




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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-828050729


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/138006/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827422874






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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617988547



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java
##########
@@ -23,17 +23,68 @@
 /**
  * Interface for a function that produces a result value for each input row.
  * <p>
- * For each input row, Spark will call a produceResult method that corresponds to the
- * {@link #inputTypes() input data types}. The expected JVM argument types must be the types used by
- * Spark's InternalRow API. If no direct method is found or when not using codegen, Spark will call
- * {@link #produceResult(InternalRow)}.
+ * To evaluate each input row, Spark will first try to lookup and use a "magic method" (described
+ * below) through Java reflection. If the method is not found, Spark will call
+ * {@link #produceResult(InternalRow)} as a fallback approach.
  * <p>
  * The JVM type of result values produced by this function must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.
+ * <p>
+ * <b>IMPORTANT</b>: the default implementation of {@link #produceResult} throws
+ * {@link UnsupportedOperationException}. Users can choose to override this method, or implement
+ * a "magic method" with name {@link #MAGIC_METHOD_NAME} which takes individual parameters
+ * instead of a {@link InternalRow}. The magic method will be loaded by Spark through Java
+ * reflection and will also provide better performance in general, due to optimizations such as
+ * codegen, removal of Java boxing, etc.
+ *
+ * For example, a scalar UDF for adding two integers can be defined as follow with the magic
+ * method approach:
+ *
+ * <pre>
+ *   public class IntegerAdd implements{@code ScalarFunction<Integer>} {
+ *     public int invoke(int left, int right) {
+ *       return left + right;
+ *     }
+ *   }
+ * </pre>
+ * In this case, since {@link #MAGIC_METHOD_NAME} is defined, Spark will use it over
+ * {@link #produceResult} to evalaute the inputs. In general Spark looks up the magic method by
+ * first converting the actual input SQL data types to their corresponding Java types following
+ * the mapping defined below, and then checking if there is a matching method from all the
+ * declared methods in the UDF class, using method name (i.e., {@link #MAGIC_METHOD_NAME}) and
+ * the Java types. If no magic method is found, Spark will falls back to use {@link #produceResult}.
+ * <p>
+ * The following are the mapping from {@link DataType SQL data type} to Java type through
+ * the magic method approach:
+ * <ul>
+ *   <li>{@link org.apache.spark.sql.types.BooleanType}: {@code boolean}</li>
+ *   <li>{@link org.apache.spark.sql.types.ByteType}: {@code byte}</li>
+ *   <li>{@link org.apache.spark.sql.types.ShortType}: {@code short}</li>
+ *   <li>{@link org.apache.spark.sql.types.IntegerType}: {@code int}</li>
+ *   <li>{@link org.apache.spark.sql.types.LongType}: {@code long}</li>
+ *   <li>{@link org.apache.spark.sql.types.FloatType}: {@code float}</li>
+ *   <li>{@link org.apache.spark.sql.types.DoubleType}: {@code double}</li>
+ *   <li>{@link org.apache.spark.sql.types.StringType}:
+ *       {@link org.apache.spark.unsafe.types.UTF8String}</li>
+ *   <li>{@link org.apache.spark.sql.types.DateType}: {@code int}</li>
+ *   <li>{@link org.apache.spark.sql.types.TimestampType}: {@code long}</li>
+ *   <li>{@link org.apache.spark.sql.types.BinaryType}: {@code byte[]}</li>
+ *   <li>{@link org.apache.spark.sql.types.DayTimeIntervalType}: {@code long}</li>
+ *   <li>{@link org.apache.spark.sql.types.YearMonthIntervalType}: {@code int}</li>
+ *   <li>{@link org.apache.spark.sql.types.DecimalType}:
+ *       {@link org.apache.spark.sql.types.Decimal}</li>
+ *   <li>{@link org.apache.spark.sql.types.StructType}: {@link InternalRow}</li>
+ *   <li>{@link org.apache.spark.sql.types.ArrayType}:
+ *       {@link org.apache.spark.sql.catalyst.util.ArrayData}</li>
+ *   <li>{@link org.apache.spark.sql.types.MapType}:
+ *       {@link org.apache.spark.sql.catalyst.util.MapData}</li>
+ *   <li>any other type: {@code Object}</li>
+ * </ul>
  *
  * @param <R> the JVM type of result values
  */
 public interface ScalarFunction<R> extends BoundFunction {
+  String MAGIC_METHOD_NAME = "invoke";

Review comment:
       I thought about that initially, but since `StructType` maps to `InternalRow`, we need a way to differentiate a) magic method with a single parameter of `StructType` and b) the default non-magic method. Users can only define the latter in this situation but in Spark we'll lookup magic method first as it has higher priority. This may cause some issue.




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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r620410980



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1955,20 +1959,29 @@ class Analyzer(override val catalogManager: CatalogManager)
    * @see https://issues.apache.org/jira/browse/SPARK-19737
    */
   object LookupFunctions extends Rule[LogicalPlan] {
+    import CatalogV2Implicits._
     override def apply(plan: LogicalPlan): LogicalPlan = {
       val externalFunctionNameSet = new mutable.HashSet[FunctionIdentifier]()
       plan.resolveExpressions {
-        case f: UnresolvedFunction
-          if externalFunctionNameSet.contains(normalizeFuncName(f.name)) => f
-        case f: UnresolvedFunction if v1SessionCatalog.isRegisteredFunction(f.name) => f
-        case f: UnresolvedFunction if v1SessionCatalog.isPersistentFunction(f.name) =>
-          externalFunctionNameSet.add(normalizeFuncName(f.name))
+        case f @ UnresolvedFunction(NonSessionCatalogAndIdentifier(_, _), _, _, _, _) =>

Review comment:
       This makes me feel that it's clearer to write if else, which also saves a lot of `AsFunctionIdentifier.unapply` calls.
   ```
   case f @ UnresolvedFunction(CatalogAndIdentifier(catalog, v2Ident), _, ...) =>
     if (CatalogV2Util.isSessionCatalog(catalog)) {
       // do nothing for v2 catalog, as we don't have api to check v2 function existence
     } else {
       val ident = v2Ident.asFunctionIdentifier
       if (externalFunctionNameSet.contains...) {
         // OK
       } else if ... {
         // OK
       } else if ... else ...
     }
   ```




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825310343


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42364/
   


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

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] rdblue commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617881769



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2021,88 +2032,218 @@ class Analyzer(override val catalogManager: CatalogManager)
                   name, other.getClass.getCanonicalName)
               }
             }
-          case u @ UnresolvedFunction(funcId, arguments, isDistinct, filter, ignoreNulls) =>
+
+          case u @ UnresolvedFunction(AsFunctionIdentifier(ident), arguments,
+          isDistinct, filter, ignoreNulls) =>

Review comment:
       Nit: indentation is off.




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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815352262






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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-823486844






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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r620710738



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java
##########
@@ -23,17 +23,68 @@
 /**
  * Interface for a function that produces a result value for each input row.
  * <p>
- * For each input row, Spark will call a produceResult method that corresponds to the
- * {@link #inputTypes() input data types}. The expected JVM argument types must be the types used by
- * Spark's InternalRow API. If no direct method is found or when not using codegen, Spark will call
- * {@link #produceResult(InternalRow)}.
+ * To evaluate each input row, Spark will first try to lookup and use a "magic method" (described
+ * below) through Java reflection. If the method is not found, Spark will call
+ * {@link #produceResult(InternalRow)} as a fallback approach.
  * <p>
  * The JVM type of result values produced by this function must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.
+ * <p>
+ * <b>IMPORTANT</b>: the default implementation of {@link #produceResult} throws
+ * {@link UnsupportedOperationException}. Users can choose to override this method, or implement
+ * a "magic method" with name {@link #MAGIC_METHOD_NAME} which takes individual parameters
+ * instead of a {@link InternalRow}. The magic method will be loaded by Spark through Java
+ * reflection and will also provide better performance in general, due to optimizations such as
+ * codegen, removal of Java boxing, etc.
+ *
+ * For example, a scalar UDF for adding two integers can be defined as follow with the magic
+ * method approach:
+ *
+ * <pre>
+ *   public class IntegerAdd implements{@code ScalarFunction<Integer>} {
+ *     public int invoke(int left, int right) {
+ *       return left + right;
+ *     }
+ *   }
+ * </pre>
+ * In this case, since {@link #MAGIC_METHOD_NAME} is defined, Spark will use it over
+ * {@link #produceResult} to evalaute the inputs. In general Spark looks up the magic method by
+ * first converting the actual input SQL data types to their corresponding Java types following
+ * the mapping defined below, and then checking if there is a matching method from all the
+ * declared methods in the UDF class, using method name (i.e., {@link #MAGIC_METHOD_NAME}) and
+ * the Java types. If no magic method is found, Spark will falls back to use {@link #produceResult}.
+ * <p>
+ * The following are the mapping from {@link DataType SQL data type} to Java type through
+ * the magic method approach:
+ * <ul>
+ *   <li>{@link org.apache.spark.sql.types.BooleanType}: {@code boolean}</li>
+ *   <li>{@link org.apache.spark.sql.types.ByteType}: {@code byte}</li>
+ *   <li>{@link org.apache.spark.sql.types.ShortType}: {@code short}</li>
+ *   <li>{@link org.apache.spark.sql.types.IntegerType}: {@code int}</li>
+ *   <li>{@link org.apache.spark.sql.types.LongType}: {@code long}</li>
+ *   <li>{@link org.apache.spark.sql.types.FloatType}: {@code float}</li>
+ *   <li>{@link org.apache.spark.sql.types.DoubleType}: {@code double}</li>
+ *   <li>{@link org.apache.spark.sql.types.StringType}:
+ *       {@link org.apache.spark.unsafe.types.UTF8String}</li>
+ *   <li>{@link org.apache.spark.sql.types.DateType}: {@code int}</li>
+ *   <li>{@link org.apache.spark.sql.types.TimestampType}: {@code long}</li>
+ *   <li>{@link org.apache.spark.sql.types.BinaryType}: {@code byte[]}</li>
+ *   <li>{@link org.apache.spark.sql.types.DayTimeIntervalType}: {@code long}</li>
+ *   <li>{@link org.apache.spark.sql.types.YearMonthIntervalType}: {@code int}</li>
+ *   <li>{@link org.apache.spark.sql.types.DecimalType}:
+ *       {@link org.apache.spark.sql.types.Decimal}</li>
+ *   <li>{@link org.apache.spark.sql.types.StructType}: {@link InternalRow}</li>
+ *   <li>{@link org.apache.spark.sql.types.ArrayType}:
+ *       {@link org.apache.spark.sql.catalyst.util.ArrayData}</li>
+ *   <li>{@link org.apache.spark.sql.types.MapType}:
+ *       {@link org.apache.spark.sql.catalyst.util.MapData}</li>
+ *   <li>any other type: {@code Object}</li>

Review comment:
       we don't need this - will remove.




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-822887222






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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r621542001



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java
##########
@@ -23,17 +23,67 @@
 /**
  * Interface for a function that produces a result value for each input row.
  * <p>
- * For each input row, Spark will call a produceResult method that corresponds to the
- * {@link #inputTypes() input data types}. The expected JVM argument types must be the types used by
- * Spark's InternalRow API. If no direct method is found or when not using codegen, Spark will call
- * {@link #produceResult(InternalRow)}.
+ * To evaluate each input row, Spark will first try to lookup and use a "magic method" (described
+ * below) through Java reflection. If the method is not found, Spark will call
+ * {@link #produceResult(InternalRow)} as a fallback approach.
  * <p>
  * The JVM type of result values produced by this function must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.
+ * <p>
+ * <b>IMPORTANT</b>: the default implementation of {@link #produceResult} throws
+ * {@link UnsupportedOperationException}. Users can choose to override this method, or implement
+ * a "magic method" with name {@link #MAGIC_METHOD_NAME} which takes individual parameters
+ * instead of a {@link InternalRow}. The magic method will be loaded by Spark through Java
+ * reflection and will also provide better performance in general, due to optimizations such as
+ * codegen, removal of Java boxing, etc.
+ *
+ * For example, a scalar UDF for adding two integers can be defined as follow with the magic
+ * method approach:
+ *
+ * <pre>
+ *   public class IntegerAdd implements{@code ScalarFunction<Integer>} {
+ *     public int invoke(int left, int right) {

Review comment:
       @cloud-fan I think we can also consider adding another "static invoke" API for those stateless UDFs. From the benchmark you did sometime back it seems this can give a decent performance improvement. WDYT?
   ```
   Java HotSpot(TM) 64-Bit Server VM 1.8.0_161-b12 on Mac OS X 10.14.6
   Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz
   UDF perf:                                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
   ------------------------------------------------------------------------------------------------------------------------
   native add                                        14206          14516         535         70.4          14.2       1.0X
   udf add                                           24609          25271         898         40.6          24.6       0.6X
   new udf add                                       18657          19096         726         53.6          18.7       0.8X
   new row udf add                                   21128          22343        1478         47.3          21.1       0.7X
   static udf add                                    16678          16887         278         60.0          16.7       0.9X
   ```




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825308922


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42364/
   


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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815394271


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/41619/
   


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

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] rdblue commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617874994



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
##########
@@ -1858,7 +1845,27 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
     }
   }
 
+  protected def getFunctionMultiparts(ctx: FunctionNameContext): Seq[String] = {
+    if (ctx.qualifiedName != null) {
+      ctx.qualifiedName().identifier().asScala.map(_.getText).toSeq
+    } else {
+      Seq(ctx.getText)
+    }
+  }
+
   /**
+   * Create a function database (optional) and name pair.
+   */
+  private def visitFunctionName(ctx: ParserRuleContext, texts: Seq[String]): FunctionIdentifier = {

Review comment:
       It looks like this function was just moved. Can you move it back to avoid commit conflicts?




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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-821672079


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137495/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-822867284


   **[Test build #137656 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137656/testReport)** for PR 32082 at commit [`887fb59`](https://github.com/apache/spark/commit/887fb59bf61356624b8819a4faf40b97df939548).


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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815394271


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/41619/
   


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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827606682


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137994/
   


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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617367921



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2021,88 +2032,218 @@ class Analyzer(override val catalogManager: CatalogManager)
                   name, other.getClass.getCanonicalName)
               }
             }
-          case u @ UnresolvedFunction(funcId, arguments, isDistinct, filter, ignoreNulls) =>
+
+          case u @ UnresolvedFunction(AsFunctionIdentifier(ident), arguments,
+          isDistinct, filter, ignoreNulls) =>
             withPosition(u) {
-              v1SessionCatalog.lookupFunction(funcId, arguments) match {
-                // AggregateWindowFunctions are AggregateFunctions that can only be evaluated within
-                // the context of a Window clause. They do not need to be wrapped in an
-                // AggregateExpression.
-                case wf: AggregateWindowFunction =>
-                  if (isDistinct) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      wf.prettyName, "DISTINCT")
-                  } else if (filter.isDefined) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      wf.prettyName, "FILTER clause")
-                  } else if (ignoreNulls) {
-                    wf match {
-                      case nthValue: NthValue =>
-                        nthValue.copy(ignoreNulls = ignoreNulls)
-                      case _ =>
-                        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                          wf.prettyName, "IGNORE NULLS")
-                    }
-                  } else {
-                    wf
-                  }
-                case owf: FrameLessOffsetWindowFunction =>
-                  if (isDistinct) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      owf.prettyName, "DISTINCT")
-                  } else if (filter.isDefined) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      owf.prettyName, "FILTER clause")
-                  } else if (ignoreNulls) {
-                    owf match {
-                      case lead: Lead =>
-                        lead.copy(ignoreNulls = ignoreNulls)
-                      case lag: Lag =>
-                        lag.copy(ignoreNulls = ignoreNulls)
-                    }
-                  } else {
-                    owf
-                  }
-                // We get an aggregate function, we need to wrap it in an AggregateExpression.
-                case agg: AggregateFunction =>
-                  if (filter.isDefined && !filter.get.deterministic) {
-                    throw QueryCompilationErrors.nonDeterministicFilterInAggregateError
-                  }
-                  if (ignoreNulls) {
-                    val aggFunc = agg match {
-                      case first: First => first.copy(ignoreNulls = ignoreNulls)
-                      case last: Last => last.copy(ignoreNulls = ignoreNulls)
-                      case _ =>
-                        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                          agg.prettyName, "IGNORE NULLS")
-                    }
-                    AggregateExpression(aggFunc, Complete, isDistinct, filter)
-                  } else {
-                    AggregateExpression(agg, Complete, isDistinct, filter)
-                  }
-                // This function is not an aggregate function, just return the resolved one.
-                case other if isDistinct =>
-                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                    other.prettyName, "DISTINCT")
-                case other if filter.isDefined =>
-                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                    other.prettyName, "FILTER clause")
-                case other if ignoreNulls =>
-                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                    other.prettyName, "IGNORE NULLS")
-                case e: String2TrimExpression if arguments.size == 2 =>
-                  if (trimWarningEnabled.get) {
-                    log.warn("Two-parameter TRIM/LTRIM/RTRIM function signatures are deprecated." +
-                      " Use SQL syntax `TRIM((BOTH | LEADING | TRAILING)? trimStr FROM str)`" +
-                      " instead.")
-                    trimWarningEnabled.set(false)
-                  }
-                  e
-                case other =>
-                  other
+              processFunctionExpr(v1SessionCatalog.lookupFunction(ident, arguments),
+                arguments, isDistinct, filter, ignoreNulls)
+            }
+
+          case u @ UnresolvedFunction(parts, arguments, isDistinct, filter, ignoreNulls) =>
+            withPosition(u) {
+              // resolve built-in or temporary functions with v2 catalog
+              val resultExpression = if (parts.length == 1) {
+                v1SessionCatalog.lookupBuiltinOrTempFunction(parts.head, arguments).map(
+                  processFunctionExpr(_, arguments, isDistinct, filter, ignoreNulls)
+                )
+              } else {
+                None
               }
+
+              resultExpression.getOrElse(
+                expandIdentifier(parts) match {
+                  case NonSessionCatalogAndIdentifier(catalog: FunctionCatalog, ident) =>
+                    lookupV2Function(catalog, ident, arguments, isDistinct, filter, ignoreNulls)
+                  case _ => u
+                }
+              )
             }
         }
     }
+
+    /**
+     * Check if the input `fn` implements the given `methodName`. If `inputType` is set, it also
+     * tries to match it against the declared parameter types.
+     */
+    private def findMethod(
+        fn: BoundFunction,
+        methodName: String,
+        inputTypeOpt: Option[Seq[DataType]] = None): Option[Method] = {
+      val cls = fn.getClass
+      inputTypeOpt match {
+        case Some(inputType) =>
+          try {
+            val argClasses = inputType.map(ScalaReflection.dataTypeJavaClass)
+            Some(cls.getDeclaredMethod(methodName, argClasses: _*))
+          } catch {
+            case _: NoSuchMethodException =>
+              None
+          }
+        case None =>
+          cls.getDeclaredMethods.find(_.getName == methodName)
+      }
+    }
+
+    private def processFunctionExpr(
+        expr: Expression,
+        arguments: Seq[Expression],
+        isDistinct: Boolean,
+        filter: Option[Expression],
+        ignoreNulls: Boolean): Expression = expr match {
+      // AggregateWindowFunctions are AggregateFunctions that can only be evaluated within
+      // the context of a Window clause. They do not need to be wrapped in an
+      // AggregateExpression.
+      case wf: AggregateWindowFunction =>
+        if (isDistinct) {
+          throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+            wf.prettyName, "DISTINCT")
+        } else if (filter.isDefined) {
+          throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+            wf.prettyName, "FILTER clause")
+        } else if (ignoreNulls) {
+          wf match {
+            case nthValue: NthValue =>
+              nthValue.copy(ignoreNulls = ignoreNulls)
+            case _ =>
+              throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+                wf.prettyName, "IGNORE NULLS")
+          }
+        } else {
+          wf
+        }
+      case owf: FrameLessOffsetWindowFunction =>
+        if (isDistinct) {
+          throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+            owf.prettyName, "DISTINCT")
+        } else if (filter.isDefined) {
+          throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+            owf.prettyName, "FILTER clause")
+        } else if (ignoreNulls) {
+          owf match {
+            case lead: Lead =>
+              lead.copy(ignoreNulls = ignoreNulls)
+            case lag: Lag =>
+              lag.copy(ignoreNulls = ignoreNulls)
+          }
+        } else {
+          owf
+        }
+      // We get an aggregate function, we need to wrap it in an AggregateExpression.
+      case agg: AggregateFunction =>
+        if (filter.isDefined && !filter.get.deterministic) {
+          throw QueryCompilationErrors.nonDeterministicFilterInAggregateError
+        }
+        if (ignoreNulls) {
+          val aggFunc = agg match {
+            case first: First => first.copy(ignoreNulls = ignoreNulls)
+            case last: Last => last.copy(ignoreNulls = ignoreNulls)
+            case _ =>
+              throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+                agg.prettyName, "IGNORE NULLS")
+          }
+          AggregateExpression(aggFunc, Complete, isDistinct, filter)
+        } else {
+          AggregateExpression(agg, Complete, isDistinct, filter)
+        }
+      // This function is not an aggregate function, just return the resolved one.
+      case other if isDistinct =>
+        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+          other.prettyName, "DISTINCT")
+      case other if filter.isDefined =>
+        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+          other.prettyName, "FILTER clause")
+      case other if ignoreNulls =>
+        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+          other.prettyName, "IGNORE NULLS")
+      case e: String2TrimExpression if arguments.size == 2 =>
+        if (trimWarningEnabled.get) {
+          log.warn("Two-parameter TRIM/LTRIM/RTRIM function signatures are deprecated." +
+            " Use SQL syntax `TRIM((BOTH | LEADING | TRAILING)? trimStr FROM str)`" +
+            " instead.")
+          trimWarningEnabled.set(false)
+        }
+        e
+      case other =>
+        other
+    }
+
+    private def lookupV2Function(
+        catalog: FunctionCatalog,
+        ident: Identifier,
+        arguments: Seq[Expression],
+        isDistinct: Boolean,
+        filter: Option[Expression],
+        ignoreNulls: Boolean): Expression = {
+      val unbound = catalog.loadFunction(ident)
+      val inputType = StructType(arguments.zipWithIndex.map {
+        case (exp, pos) => StructField(s"_$pos", exp.dataType, exp.nullable)
+      })
+      val bound = try {
+        unbound.bind(inputType)
+      } catch {
+        case unsupported: UnsupportedOperationException =>
+          failAnalysis(s"Function '${unbound.name}' cannot process input: " +

Review comment:
       shall we construct `AnalysisException` directly here and set the cause?




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

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] SparkQA removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827863630


   **[Test build #138006 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138006/testReport)** for PR 32082 at commit [`c18715f`](https://github.com/apache/spark/commit/c18715fcc304f4a805d420e193d4a3884b3f7e4f).


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

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] rdblue commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617867877



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java
##########
@@ -23,17 +23,68 @@
 /**
  * Interface for a function that produces a result value for each input row.
  * <p>
- * For each input row, Spark will call a produceResult method that corresponds to the
- * {@link #inputTypes() input data types}. The expected JVM argument types must be the types used by
- * Spark's InternalRow API. If no direct method is found or when not using codegen, Spark will call
- * {@link #produceResult(InternalRow)}.
+ * To evaluate each input row, Spark will first try to lookup and use a "magic method" (described
+ * below) through Java reflection. If the method is not found, Spark will call
+ * {@link #produceResult(InternalRow)} as a fallback approach.
  * <p>
  * The JVM type of result values produced by this function must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.
+ * <p>
+ * <b>IMPORTANT</b>: the default implementation of {@link #produceResult} throws
+ * {@link UnsupportedOperationException}. Users can choose to override this method, or implement
+ * a "magic method" with name {@link #MAGIC_METHOD_NAME} which takes individual parameters
+ * instead of a {@link InternalRow}. The magic method will be loaded by Spark through Java
+ * reflection and will also provide better performance in general, due to optimizations such as
+ * codegen, removal of Java boxing, etc.
+ *
+ * For example, a scalar UDF for adding two integers can be defined as follow with the magic
+ * method approach:
+ *
+ * <pre>
+ *   public class IntegerAdd implements{@code ScalarFunction<Integer>} {
+ *     public int invoke(int left, int right) {
+ *       return left + right;
+ *     }
+ *   }
+ * </pre>
+ * In this case, since {@link #MAGIC_METHOD_NAME} is defined, Spark will use it over
+ * {@link #produceResult} to evalaute the inputs. In general Spark looks up the magic method by
+ * first converting the actual input SQL data types to their corresponding Java types following
+ * the mapping defined below, and then checking if there is a matching method from all the
+ * declared methods in the UDF class, using method name (i.e., {@link #MAGIC_METHOD_NAME}) and
+ * the Java types. If no magic method is found, Spark will falls back to use {@link #produceResult}.
+ * <p>
+ * The following are the mapping from {@link DataType SQL data type} to Java type through
+ * the magic method approach:
+ * <ul>
+ *   <li>{@link org.apache.spark.sql.types.BooleanType}: {@code boolean}</li>
+ *   <li>{@link org.apache.spark.sql.types.ByteType}: {@code byte}</li>
+ *   <li>{@link org.apache.spark.sql.types.ShortType}: {@code short}</li>
+ *   <li>{@link org.apache.spark.sql.types.IntegerType}: {@code int}</li>
+ *   <li>{@link org.apache.spark.sql.types.LongType}: {@code long}</li>
+ *   <li>{@link org.apache.spark.sql.types.FloatType}: {@code float}</li>
+ *   <li>{@link org.apache.spark.sql.types.DoubleType}: {@code double}</li>
+ *   <li>{@link org.apache.spark.sql.types.StringType}:
+ *       {@link org.apache.spark.unsafe.types.UTF8String}</li>
+ *   <li>{@link org.apache.spark.sql.types.DateType}: {@code int}</li>
+ *   <li>{@link org.apache.spark.sql.types.TimestampType}: {@code long}</li>
+ *   <li>{@link org.apache.spark.sql.types.BinaryType}: {@code byte[]}</li>
+ *   <li>{@link org.apache.spark.sql.types.DayTimeIntervalType}: {@code long}</li>
+ *   <li>{@link org.apache.spark.sql.types.YearMonthIntervalType}: {@code int}</li>
+ *   <li>{@link org.apache.spark.sql.types.DecimalType}:
+ *       {@link org.apache.spark.sql.types.Decimal}</li>
+ *   <li>{@link org.apache.spark.sql.types.StructType}: {@link InternalRow}</li>
+ *   <li>{@link org.apache.spark.sql.types.ArrayType}:
+ *       {@link org.apache.spark.sql.catalyst.util.ArrayData}</li>
+ *   <li>{@link org.apache.spark.sql.types.MapType}:
+ *       {@link org.apache.spark.sql.catalyst.util.MapData}</li>
+ *   <li>any other type: {@code Object}</li>
+ * </ul>
  *
  * @param <R> the JVM type of result values
  */
 public interface ScalarFunction<R> extends BoundFunction {
+  String MAGIC_METHOD_NAME = "invoke";

Review comment:
       Why is this needed? I think that the magic name should be "produceResult" just like the `InternalRow` version so that it is clear what the method is supposed to 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.

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r622387401



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java
##########
@@ -23,17 +23,67 @@
 /**
  * Interface for a function that produces a result value for each input row.
  * <p>
- * For each input row, Spark will call a produceResult method that corresponds to the
- * {@link #inputTypes() input data types}. The expected JVM argument types must be the types used by
- * Spark's InternalRow API. If no direct method is found or when not using codegen, Spark will call
- * {@link #produceResult(InternalRow)}.
+ * To evaluate each input row, Spark will first try to lookup and use a "magic method" (described
+ * below) through Java reflection. If the method is not found, Spark will call
+ * {@link #produceResult(InternalRow)} as a fallback approach.
  * <p>
  * The JVM type of result values produced by this function must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.
+ * <p>
+ * <b>IMPORTANT</b>: the default implementation of {@link #produceResult} throws
+ * {@link UnsupportedOperationException}. Users can choose to override this method, or implement
+ * a "magic method" with name {@link #MAGIC_METHOD_NAME} which takes individual parameters
+ * instead of a {@link InternalRow}. The magic method will be loaded by Spark through Java
+ * reflection and will also provide better performance in general, due to optimizations such as
+ * codegen, removal of Java boxing, etc.
+ *
+ * For example, a scalar UDF for adding two integers can be defined as follow with the magic
+ * method approach:
+ *
+ * <pre>
+ *   public class IntegerAdd implements{@code ScalarFunction<Integer>} {
+ *     public int invoke(int left, int right) {

Review comment:
       yea




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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-821725854


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137496/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825415631


   **[Test build #137848 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137848/testReport)** for PR 32082 at commit [`47bb15d`](https://github.com/apache/spark/commit/47bb15dbe2a84ff536c1b1b3eb4f6e9a0a3f7615).


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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825649215


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137851/
   


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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815374758






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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815656588


   **[Test build #137064 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137064/testReport)** for PR 32082 at commit [`173f7e1`](https://github.com/apache/spark/commit/173f7e1ca6c1dece9a89655f8c8a8a29ff840a15).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public 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.

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825187750






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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825313308


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42364/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-821614413


   **[Test build #137495 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137495/testReport)** for PR 32082 at commit [`f949f84`](https://github.com/apache/spark/commit/f949f84d9b07daa2009ac32b987f09584ce6c521).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public 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.

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r614082282



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/V2Aggregator.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.connector.catalog.functions.{AggregateFunction => V2AggregateFunction}
+import org.apache.spark.sql.types.DataType
+
+case class V2Aggregator[BUF <: java.io.Serializable, OUT](
+    aggrFunc: V2AggregateFunction[BUF, OUT],
+    children: Seq[Expression],
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0) extends TypedImperativeAggregate[BUF] {
+  override def createAggregationBuffer(): BUF = aggrFunc.newAggregationState()
+
+  override def update(buffer: BUF, input: InternalRow): BUF = aggrFunc.update(buffer, input)
+
+  override def merge(buffer: BUF, input: BUF): BUF = aggrFunc.merge(buffer, input)

Review comment:
       ditto




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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-818191606


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/41817/
   


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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r614062055



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java
##########
@@ -30,10 +30,71 @@
  * <p>
  * The JVM type of result values produced by this function must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.
+ * <p>
+ * <b>IMPORTANT</b>: the default implementation of {@link #produceResult} throws
+ * {@link UnsupportedOperationException}. Users can choose to override this method, or implement
+ * a "magic method" with name {@link #MAGIC_METHOD_NAME} which takes individual parameters

Review comment:
       This conflicts with the doc above which says
   ```
    * For each input row, Spark will call a produceResult method that corresponds to the
    * {@link #inputTypes() input data types}.
   ```
   
   Shall we update the doc above?




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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r614070471



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2025,9 +2030,74 @@ class Analyzer(override val catalogManager: CatalogManager)
                   name, other.getClass.getCanonicalName)
               }
             }
-          case u @ UnresolvedFunction(funcId, arguments, isDistinct, filter, ignoreNulls) =>
+          case UnresolvedFunction(NonSessionCatalogAndIdentifier(v2Catalog, ident), arguments,
+            isDistinct, filter, _) if v2Catalog.isFunctionCatalog =>
+            val unbound = v2Catalog.asFunctionCatalog.loadFunction(ident)
+
+            val inputType = StructType(arguments.zipWithIndex.map {
+              case (exp, pos) => StructField(s"_$pos", exp.dataType, exp.nullable)
+            })
+
+            val bound = try {
+              unbound.bind(inputType)
+            } catch {
+              case unsupported: UnsupportedOperationException =>
+                failAnalysis(s"Function '${unbound.name}' cannot process input: " +
+                  s"(${arguments.map(_.dataType.simpleString).mkString(", ")}): " +
+                  unsupported.getMessage)
+            }
+
+            bound match {
+              case scalarFunc: ScalarFunction[_] =>
+                if (isDistinct) {
+                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+                    scalarFunc.name(), "DISTINCT")
+                } else if (filter.isDefined) {
+                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+                    scalarFunc.name(), "FILTER clause")
+                } else {
+                  // TODO: implement type coercion by looking at input type from the UDF. We may
+                  //  also want to check if the parameter types from the magic method match the
+                  //  input type through `BoundFunction.inputTypes`.
+                  val argClasses = inputType.fields.map(_.dataType)
+                  findMethod(scalarFunc, ScalarFunction.MAGIC_METHOD_NAME, Some(argClasses)) match {
+                    case Some(_) =>
+                      val caller = Literal.create(scalarFunc, ObjectType(scalarFunc.getClass))
+                      Invoke(caller, ScalarFunction.MAGIC_METHOD_NAME, scalarFunc.resultType(),
+                        arguments, returnNullable = scalarFunc.isResultNullable)
+                    case _ =>
+                      // TODO: handle functions defined in Scala too - in Scala, even if a
+                      //  subclass do not override the default method in parent interface defined
+                      //  in Java, the method can still be found from `getDeclaredMethod`.
+                      findMethod(scalarFunc, "produceResult", Some(Seq(inputType))) match {

Review comment:
       I'm a bit confused. `produceResult` is defined in the interface. Why do we need reflection here?




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825173918


   **[Test build #137822 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137822/testReport)** for PR 32082 at commit [`c31a70b`](https://github.com/apache/spark/commit/c31a70bed54e00005a5fd2bbd4d5f4da9a17ec4e).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public 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.

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r614063088



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java
##########
@@ -30,10 +30,71 @@
  * <p>
  * The JVM type of result values produced by this function must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.
+ * <p>
+ * <b>IMPORTANT</b>: the default implementation of {@link #produceResult} throws
+ * {@link UnsupportedOperationException}. Users can choose to override this method, or implement
+ * a "magic method" with name {@link #MAGIC_METHOD_NAME} which takes individual parameters
+ * instead of a {@link InternalRow}. The magic method will be loaded by Spark through Java
+ * reflection and also will provide better performance in general, due to optimizations such as
+ * codegen, Java boxing and so on.
+ *
+ * For example, a scalar UDF for adding two integers can be defined as follow with the magic
+ * method approach:
+ *
+ * <pre>
+ *   public class IntegerAdd implements{@code ScalarFunction<Integer>} {
+ *     public int invoke(int left, int right) {
+ *       return left + right;
+ *     }
+ *
+ *    {@literal @}Override
+ *     public produceResult(InternalRow input) {

Review comment:
       Most of the time, people can skip implementing `produceResult`?




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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r615600247



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java
##########
@@ -30,10 +30,71 @@
  * <p>
  * The JVM type of result values produced by this function must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.
+ * <p>
+ * <b>IMPORTANT</b>: the default implementation of {@link #produceResult} throws
+ * {@link UnsupportedOperationException}. Users can choose to override this method, or implement
+ * a "magic method" with name {@link #MAGIC_METHOD_NAME} which takes individual parameters
+ * instead of a {@link InternalRow}. The magic method will be loaded by Spark through Java
+ * reflection and also will provide better performance in general, due to optimizations such as
+ * codegen, Java boxing and so on.
+ *
+ * For example, a scalar UDF for adding two integers can be defined as follow with the magic
+ * method approach:
+ *
+ * <pre>
+ *   public class IntegerAdd implements{@code ScalarFunction<Integer>} {
+ *     public int invoke(int left, int right) {
+ *       return left + right;
+ *     }
+ *
+ *    {@literal @}Override
+ *     public produceResult(InternalRow input) {
+ *       int left = input.getInt(0);
+ *       int right = input.getInt(1);
+ *       return left + right;
+ *     }
+ *   }
+ * </pre>
+ * In this case, both {@link #MAGIC_METHOD_NAME} and {@link #produceResult} are defined, and Spark
+ * will first lookup the {@link #MAGIC_METHOD_NAME} method during query analysis. This is done by
+ * first converting the actual input SQL data types to their corresponding Java types following the
+ * mapping defined below, and then checking if there is a matching method from all the declared
+ * methods in the UDF class, using method name (i.e., {@link #MAGIC_METHOD_NAME}) and the Java
+ * types. If no magic method is found, Spark will falls back to use {@link #produceResult}.
+ * <p>
+ * The following are the mapping from {@link DataType SQL data type} to Java type through
+ * the magic method approach:
+ * <ul>
+ *   <li>{@link org.apache.spark.sql.types.BooleanType}: {@code boolean}</li>
+ *   <li>{@link org.apache.spark.sql.types.ByteType}: {@code byte}</li>
+ *   <li>{@link org.apache.spark.sql.types.ShortType}: {@code short}</li>
+ *   <li>{@link org.apache.spark.sql.types.IntegerType}: {@code int}</li>
+ *   <li>{@link org.apache.spark.sql.types.LongType}: {@code long}</li>
+ *   <li>{@link org.apache.spark.sql.types.FloatType}: {@code float}</li>
+ *   <li>{@link org.apache.spark.sql.types.DoubleType}: {@code double}</li>
+ *   <li>{@link org.apache.spark.sql.types.StringType}:
+ *       {@link org.apache.spark.unsafe.types.UTF8String}</li>
+ *   <li>{@link org.apache.spark.sql.types.DateType}: {@code int}</li>
+ *   <li>{@link org.apache.spark.sql.types.TimestampType}: {@code long}</li>
+ *   <li>{@link org.apache.spark.sql.types.BinaryType}: {@code byte[]}</li>
+ *   <li>{@link org.apache.spark.sql.types.CalendarIntervalType}:

Review comment:
       See https://issues.apache.org/jira/browse/SPARK-27790
   
   I think we will deprecate it soon. cc @MaxGekk 




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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815675132


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137064/
   


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

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] SparkQA removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825281701


   **[Test build #137834 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137834/testReport)** for PR 32082 at commit [`55c1f4b`](https://github.com/apache/spark/commit/55c1f4bcd72ffbd7000b2304833af21c0e560772).


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827384282






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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825649215


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137851/
   


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

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] SparkQA removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-823449486


   **[Test build #137699 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137699/testReport)** for PR 32082 at commit [`b08c814`](https://github.com/apache/spark/commit/b08c814f51daca6fb34fe862143fffe4a6e3a3d1).


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

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] SparkQA removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-817994057


   **[Test build #137227 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137227/testReport)** for PR 32082 at commit [`dc9dde6`](https://github.com/apache/spark/commit/dc9dde65db5a815460e9e430acc4f07554ed0dcd).


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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r621542001



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java
##########
@@ -23,17 +23,67 @@
 /**
  * Interface for a function that produces a result value for each input row.
  * <p>
- * For each input row, Spark will call a produceResult method that corresponds to the
- * {@link #inputTypes() input data types}. The expected JVM argument types must be the types used by
- * Spark's InternalRow API. If no direct method is found or when not using codegen, Spark will call
- * {@link #produceResult(InternalRow)}.
+ * To evaluate each input row, Spark will first try to lookup and use a "magic method" (described
+ * below) through Java reflection. If the method is not found, Spark will call
+ * {@link #produceResult(InternalRow)} as a fallback approach.
  * <p>
  * The JVM type of result values produced by this function must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.
+ * <p>
+ * <b>IMPORTANT</b>: the default implementation of {@link #produceResult} throws
+ * {@link UnsupportedOperationException}. Users can choose to override this method, or implement
+ * a "magic method" with name {@link #MAGIC_METHOD_NAME} which takes individual parameters
+ * instead of a {@link InternalRow}. The magic method will be loaded by Spark through Java
+ * reflection and will also provide better performance in general, due to optimizations such as
+ * codegen, removal of Java boxing, etc.
+ *
+ * For example, a scalar UDF for adding two integers can be defined as follow with the magic
+ * method approach:
+ *
+ * <pre>
+ *   public class IntegerAdd implements{@code ScalarFunction<Integer>} {
+ *     public int invoke(int left, int right) {

Review comment:
       @cloud-fan I think we can also consider adding another "static invoke" API for those stateless UDFs. From the benchmark you did sometime back it seems this can give a decent performance improvements. WDYT?
   ```
   Java HotSpot(TM) 64-Bit Server VM 1.8.0_161-b12 on Mac OS X 10.14.6
   Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz
   UDF perf:                                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
   ------------------------------------------------------------------------------------------------------------------------
   native add                                        14206          14516         535         70.4          14.2       1.0X
   udf add                                           24609          25271         898         40.6          24.6       0.6X
   new udf add                                       18657          19096         726         53.6          18.7       0.8X
   new row udf add                                   21128          22343        1478         47.3          21.1       0.7X
   static udf add                                    16678          16887         278         60.0          16.7       0.9X
   ```




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

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] rdblue commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617876109



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala
##########
@@ -154,6 +154,26 @@ private[sql] trait LookupCatalog extends Logging {
     }
   }
 
+  object AsFunctionIdentifier {
+    def unapply(parts: Seq[String]): Option[FunctionIdentifier] = {
+      def namesToFunctionIdentifier(names: Seq[String]): Option[FunctionIdentifier] = names match {
+        case Seq(name) => Some(FunctionIdentifier(name))
+        case Seq(database, name) => Some(FunctionIdentifier(name, Some(database)))
+        case _ => None
+      }
+      parts match {
+        case CatalogAndMultipartIdentifier(None, names)
+          if CatalogV2Util.isSessionCatalog(currentCatalog) =>
+          namesToFunctionIdentifier(names)
+        case CatalogAndMultipartIdentifier(Some(catalog), names)
+          if CatalogV2Util.isSessionCatalog(catalog) &&
+            CatalogV2Util.isSessionCatalog(currentCatalog) =>

Review comment:
       Why is the current catalog also checked? Can't I refer to `spark_catalog.db.func` and get a `FunctionIdentifier` back?




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

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] sunchao commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-823460046


   Also cc @xkrogen @wmoustafa and @jzhuge 


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825510468


   **[Test build #137848 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137848/testReport)** for PR 32082 at commit [`47bb15d`](https://github.com/apache/spark/commit/47bb15dbe2a84ff536c1b1b3eb4f6e9a0a3f7615).
    * This patch passes all tests.
    * This patch **does not merge cleanly**.
    * This patch adds no public 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.

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827461262


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42515/
   


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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-823628524


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137699/
   


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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-822892047


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42185/
   


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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r615093716



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/V2Aggregator.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.connector.catalog.functions.{AggregateFunction => V2AggregateFunction}
+import org.apache.spark.sql.types.DataType
+
+case class V2Aggregator[BUF <: java.io.Serializable, OUT](
+    aggrFunc: V2AggregateFunction[BUF, OUT],
+    children: Seq[Expression],
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0) extends TypedImperativeAggregate[BUF] {
+  override def createAggregationBuffer(): BUF = aggrFunc.newAggregationState()
+
+  override def update(buffer: BUF, input: InternalRow): BUF = aggrFunc.update(buffer, input)
+
+  override def merge(buffer: BUF, input: BUF): BUF = aggrFunc.merge(buffer, input)

Review comment:
       I don't think we need to do anything here? the input is a `BUF`.




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815460314


   **[Test build #137041 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137041/testReport)** for PR 32082 at commit [`c522276`](https://github.com/apache/spark/commit/c522276bf1f051af6200c17bdf51a4aa0f565b0a).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public 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.

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r614290595



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/V2Aggregator.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.connector.catalog.functions.{AggregateFunction => V2AggregateFunction}
+import org.apache.spark.sql.types.DataType
+
+case class V2Aggregator[BUF <: java.io.Serializable, OUT](
+    aggrFunc: V2AggregateFunction[BUF, OUT],
+    children: Seq[Expression],
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0) extends TypedImperativeAggregate[BUF] {
+  override def createAggregationBuffer(): BUF = aggrFunc.newAggregationState()
+
+  override def update(buffer: BUF, input: InternalRow): BUF = aggrFunc.update(buffer, input)

Review comment:
       Oops you are right. We should evaluate the `input` first. I'll also add a test case for this.
   
   Let me also handle the magic method case for aggregate functions (was planning to do that separately).




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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-822973964


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137656/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-823608149


   **[Test build #137699 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137699/testReport)** for PR 32082 at commit [`b08c814`](https://github.com/apache/spark/commit/b08c814f51daca6fb34fe862143fffe4a6e3a3d1).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public 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.

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] SparkQA removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815513127


   **[Test build #137064 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137064/testReport)** for PR 32082 at commit [`173f7e1`](https://github.com/apache/spark/commit/173f7e1ca6c1dece9a89655f8c8a8a29ff840a15).


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827489559


   **[Test build #137987 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137987/testReport)** for PR 32082 at commit [`f25b5e6`](https://github.com/apache/spark/commit/f25b5e683f796c353296adb519b4882e37fbc213).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public 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.

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r622441347



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java
##########
@@ -23,17 +23,67 @@
 /**
  * Interface for a function that produces a result value for each input row.
  * <p>
- * For each input row, Spark will call a produceResult method that corresponds to the
- * {@link #inputTypes() input data types}. The expected JVM argument types must be the types used by
- * Spark's InternalRow API. If no direct method is found or when not using codegen, Spark will call
- * {@link #produceResult(InternalRow)}.
+ * To evaluate each input row, Spark will first try to lookup and use a "magic method" (described
+ * below) through Java reflection. If the method is not found, Spark will call
+ * {@link #produceResult(InternalRow)} as a fallback approach.
  * <p>
  * The JVM type of result values produced by this function must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.
+ * <p>
+ * <b>IMPORTANT</b>: the default implementation of {@link #produceResult} throws
+ * {@link UnsupportedOperationException}. Users can choose to override this method, or implement
+ * a "magic method" with name {@link #MAGIC_METHOD_NAME} which takes individual parameters
+ * instead of a {@link InternalRow}. The magic method will be loaded by Spark through Java
+ * reflection and will also provide better performance in general, due to optimizations such as
+ * codegen, removal of Java boxing, etc.
+ *
+ * For example, a scalar UDF for adding two integers can be defined as follow with the magic
+ * method approach:
+ *
+ * <pre>
+ *   public class IntegerAdd implements{@code ScalarFunction<Integer>} {
+ *     public int invoke(int left, int right) {

Review comment:
       Created https://issues.apache.org/jira/browse/SPARK-35261




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827378494






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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827908675


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42525/
   


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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-817999593


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137227/
   


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

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] SparkQA removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825415631


   **[Test build #137848 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137848/testReport)** for PR 32082 at commit [`47bb15d`](https://github.com/apache/spark/commit/47bb15dbe2a84ff536c1b1b3eb4f6e9a0a3f7615).


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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r614082050



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/V2Aggregator.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.connector.catalog.functions.{AggregateFunction => V2AggregateFunction}
+import org.apache.spark.sql.types.DataType
+
+case class V2Aggregator[BUF <: java.io.Serializable, OUT](
+    aggrFunc: V2AggregateFunction[BUF, OUT],
+    children: Seq[Expression],
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0) extends TypedImperativeAggregate[BUF] {
+  override def createAggregationBuffer(): BUF = aggrFunc.newAggregationState()
+
+  override def update(buffer: BUF, input: InternalRow): BUF = aggrFunc.update(buffer, input)

Review comment:
       This seems incorrect. The actual input is `children.map(_.eval(input))`, which is not an `InternalRow` and the "magic method" is useful in `V2AggregateFunction` as well.




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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r620410980



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1955,20 +1959,29 @@ class Analyzer(override val catalogManager: CatalogManager)
    * @see https://issues.apache.org/jira/browse/SPARK-19737
    */
   object LookupFunctions extends Rule[LogicalPlan] {
+    import CatalogV2Implicits._
     override def apply(plan: LogicalPlan): LogicalPlan = {
       val externalFunctionNameSet = new mutable.HashSet[FunctionIdentifier]()
       plan.resolveExpressions {
-        case f: UnresolvedFunction
-          if externalFunctionNameSet.contains(normalizeFuncName(f.name)) => f
-        case f: UnresolvedFunction if v1SessionCatalog.isRegisteredFunction(f.name) => f
-        case f: UnresolvedFunction if v1SessionCatalog.isPersistentFunction(f.name) =>
-          externalFunctionNameSet.add(normalizeFuncName(f.name))
+        case f @ UnresolvedFunction(NonSessionCatalogAndIdentifier(_, _), _, _, _, _) =>

Review comment:
       This makes me feel that it's clearer to write if else, which also saves a lot of `AsFunctionIdentifier.unapply` calls.
   ```
   case f @ UnresolvedFunction(CatalogAndIdentifier(catalog, v2Ident), _, ...) =>
     if (CatalogV2Util.isSessionCatalog(catalog)) {
       // do nothing for v2 catalog, as we don't have api to check v2 function existence
     } else {
       val ident = v2Ident.asFunctionIdentifier
       if (...) else if ... else ...
     }
   ```




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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r620410980



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1955,20 +1959,29 @@ class Analyzer(override val catalogManager: CatalogManager)
    * @see https://issues.apache.org/jira/browse/SPARK-19737
    */
   object LookupFunctions extends Rule[LogicalPlan] {
+    import CatalogV2Implicits._
     override def apply(plan: LogicalPlan): LogicalPlan = {
       val externalFunctionNameSet = new mutable.HashSet[FunctionIdentifier]()
       plan.resolveExpressions {
-        case f: UnresolvedFunction
-          if externalFunctionNameSet.contains(normalizeFuncName(f.name)) => f
-        case f: UnresolvedFunction if v1SessionCatalog.isRegisteredFunction(f.name) => f
-        case f: UnresolvedFunction if v1SessionCatalog.isPersistentFunction(f.name) =>
-          externalFunctionNameSet.add(normalizeFuncName(f.name))
+        case f @ UnresolvedFunction(NonSessionCatalogAndIdentifier(_, _), _, _, _, _) =>

Review comment:
       This makes me feel that it's clearer to write if else, which also saves a lot of `AsFunctionIdentifier.unapply` calls.
   ```
   case f @ UnresolvedFunction(CatalogAndIdentifier(catalog, v2Ident), _, ...) =>
     if (!CatalogV2Util.isSessionCatalog(catalog)) {
       // do nothing for v2 catalog, as we don't have api to check v2 function existence
     } else {
       val ident = v2Ident.asFunctionIdentifier
       if (externalFunctionNameSet.contains...) {
         // OK
       } else if ... {
         // OK
       } else if ... else ...
     }
   ```




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

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] SparkQA removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825114810


   **[Test build #137820 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137820/testReport)** for PR 32082 at commit [`66dac00`](https://github.com/apache/spark/commit/66dac00f7d417452dc7eb268475b22ddcbb40075).


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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-821672079


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137495/
   


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

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] sunchao commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-818305195


   cc @cloud-fan @rdblue 


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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r620898563



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/higherOrderFunctions.scala
##########
@@ -30,13 +30,14 @@ import org.apache.spark.sql.types.DataType
  * so we need to resolve higher order function when all children are either resolved or a lambda
  * function.
  */
-case class ResolveHigherOrderFunctions(catalog: SessionCatalog) extends Rule[LogicalPlan] {
+case class ResolveHigherOrderFunctions(catalogManager: CatalogManager)
+  extends Rule[LogicalPlan] with LookupCatalog {
 
   override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveExpressions {
-    case u @ UnresolvedFunction(fn, children, false, filter, ignoreNulls)
+    case u @ UnresolvedFunction(AsFunctionIdentifier(ident), children, false, filter, ignoreNulls)

Review comment:
       Neverwind the above. I've updated `AsFunctionIdentifier` to handle the built-in/temp function case so the logic here is much simpler 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.

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] rdblue commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617886723



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2021,88 +2032,218 @@ class Analyzer(override val catalogManager: CatalogManager)
                   name, other.getClass.getCanonicalName)
               }
             }
-          case u @ UnresolvedFunction(funcId, arguments, isDistinct, filter, ignoreNulls) =>
+
+          case u @ UnresolvedFunction(AsFunctionIdentifier(ident), arguments,
+          isDistinct, filter, ignoreNulls) =>
             withPosition(u) {
-              v1SessionCatalog.lookupFunction(funcId, arguments) match {
-                // AggregateWindowFunctions are AggregateFunctions that can only be evaluated within
-                // the context of a Window clause. They do not need to be wrapped in an
-                // AggregateExpression.
-                case wf: AggregateWindowFunction =>
-                  if (isDistinct) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      wf.prettyName, "DISTINCT")
-                  } else if (filter.isDefined) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      wf.prettyName, "FILTER clause")
-                  } else if (ignoreNulls) {
-                    wf match {
-                      case nthValue: NthValue =>
-                        nthValue.copy(ignoreNulls = ignoreNulls)
-                      case _ =>
-                        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                          wf.prettyName, "IGNORE NULLS")
-                    }
-                  } else {
-                    wf
-                  }
-                case owf: FrameLessOffsetWindowFunction =>
-                  if (isDistinct) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      owf.prettyName, "DISTINCT")
-                  } else if (filter.isDefined) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      owf.prettyName, "FILTER clause")
-                  } else if (ignoreNulls) {
-                    owf match {
-                      case lead: Lead =>
-                        lead.copy(ignoreNulls = ignoreNulls)
-                      case lag: Lag =>
-                        lag.copy(ignoreNulls = ignoreNulls)
-                    }
-                  } else {
-                    owf
-                  }
-                // We get an aggregate function, we need to wrap it in an AggregateExpression.
-                case agg: AggregateFunction =>
-                  if (filter.isDefined && !filter.get.deterministic) {
-                    throw QueryCompilationErrors.nonDeterministicFilterInAggregateError
-                  }
-                  if (ignoreNulls) {
-                    val aggFunc = agg match {
-                      case first: First => first.copy(ignoreNulls = ignoreNulls)
-                      case last: Last => last.copy(ignoreNulls = ignoreNulls)
-                      case _ =>
-                        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                          agg.prettyName, "IGNORE NULLS")
-                    }
-                    AggregateExpression(aggFunc, Complete, isDistinct, filter)
-                  } else {
-                    AggregateExpression(agg, Complete, isDistinct, filter)
-                  }
-                // This function is not an aggregate function, just return the resolved one.
-                case other if isDistinct =>
-                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                    other.prettyName, "DISTINCT")
-                case other if filter.isDefined =>
-                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                    other.prettyName, "FILTER clause")
-                case other if ignoreNulls =>
-                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                    other.prettyName, "IGNORE NULLS")
-                case e: String2TrimExpression if arguments.size == 2 =>
-                  if (trimWarningEnabled.get) {
-                    log.warn("Two-parameter TRIM/LTRIM/RTRIM function signatures are deprecated." +
-                      " Use SQL syntax `TRIM((BOTH | LEADING | TRAILING)? trimStr FROM str)`" +
-                      " instead.")
-                    trimWarningEnabled.set(false)
-                  }
-                  e
-                case other =>
-                  other
+              processFunctionExpr(v1SessionCatalog.lookupFunction(ident, arguments),
+                arguments, isDistinct, filter, ignoreNulls)
+            }
+
+          case u @ UnresolvedFunction(parts, arguments, isDistinct, filter, ignoreNulls) =>
+            withPosition(u) {
+              // resolve built-in or temporary functions with v2 catalog
+              val resultExpression = if (parts.length == 1) {
+                v1SessionCatalog.lookupBuiltinOrTempFunction(parts.head, arguments).map(
+                  processFunctionExpr(_, arguments, isDistinct, filter, ignoreNulls)
+                )
+              } else {
+                None
               }
+
+              resultExpression.getOrElse(
+                expandIdentifier(parts) match {
+                  case NonSessionCatalogAndIdentifier(catalog: FunctionCatalog, ident) =>
+                    lookupV2Function(catalog, ident, arguments, isDistinct, filter, ignoreNulls)
+                  case _ => u
+                }
+              )
             }
         }
     }
+
+    /**
+     * Check if the input `fn` implements the given `methodName`. If `inputType` is set, it also
+     * tries to match it against the declared parameter types.
+     */
+    private def findMethod(
+        fn: BoundFunction,
+        methodName: String,
+        inputTypeOpt: Option[Seq[DataType]] = None): Option[Method] = {
+      val cls = fn.getClass
+      inputTypeOpt match {
+        case Some(inputType) =>
+          try {
+            val argClasses = inputType.map(ScalaReflection.dataTypeJavaClass)
+            Some(cls.getDeclaredMethod(methodName, argClasses: _*))
+          } catch {
+            case _: NoSuchMethodException =>
+              None
+          }
+        case None =>
+          cls.getDeclaredMethods.find(_.getName == methodName)
+      }
+    }
+
+    private def processFunctionExpr(
+        expr: Expression,
+        arguments: Seq[Expression],
+        isDistinct: Boolean,
+        filter: Option[Expression],
+        ignoreNulls: Boolean): Expression = expr match {
+      // AggregateWindowFunctions are AggregateFunctions that can only be evaluated within
+      // the context of a Window clause. They do not need to be wrapped in an
+      // AggregateExpression.
+      case wf: AggregateWindowFunction =>
+        if (isDistinct) {
+          throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+            wf.prettyName, "DISTINCT")
+        } else if (filter.isDefined) {
+          throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+            wf.prettyName, "FILTER clause")
+        } else if (ignoreNulls) {
+          wf match {
+            case nthValue: NthValue =>
+              nthValue.copy(ignoreNulls = ignoreNulls)
+            case _ =>
+              throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+                wf.prettyName, "IGNORE NULLS")
+          }
+        } else {
+          wf
+        }
+      case owf: FrameLessOffsetWindowFunction =>
+        if (isDistinct) {
+          throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+            owf.prettyName, "DISTINCT")
+        } else if (filter.isDefined) {
+          throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+            owf.prettyName, "FILTER clause")
+        } else if (ignoreNulls) {
+          owf match {
+            case lead: Lead =>
+              lead.copy(ignoreNulls = ignoreNulls)
+            case lag: Lag =>
+              lag.copy(ignoreNulls = ignoreNulls)
+          }
+        } else {
+          owf
+        }
+      // We get an aggregate function, we need to wrap it in an AggregateExpression.
+      case agg: AggregateFunction =>
+        if (filter.isDefined && !filter.get.deterministic) {
+          throw QueryCompilationErrors.nonDeterministicFilterInAggregateError
+        }
+        if (ignoreNulls) {
+          val aggFunc = agg match {
+            case first: First => first.copy(ignoreNulls = ignoreNulls)
+            case last: Last => last.copy(ignoreNulls = ignoreNulls)
+            case _ =>
+              throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+                agg.prettyName, "IGNORE NULLS")
+          }
+          AggregateExpression(aggFunc, Complete, isDistinct, filter)
+        } else {
+          AggregateExpression(agg, Complete, isDistinct, filter)
+        }
+      // This function is not an aggregate function, just return the resolved one.
+      case other if isDistinct =>
+        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+          other.prettyName, "DISTINCT")
+      case other if filter.isDefined =>
+        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+          other.prettyName, "FILTER clause")
+      case other if ignoreNulls =>
+        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+          other.prettyName, "IGNORE NULLS")
+      case e: String2TrimExpression if arguments.size == 2 =>
+        if (trimWarningEnabled.get) {
+          log.warn("Two-parameter TRIM/LTRIM/RTRIM function signatures are deprecated." +
+            " Use SQL syntax `TRIM((BOTH | LEADING | TRAILING)? trimStr FROM str)`" +
+            " instead.")
+          trimWarningEnabled.set(false)
+        }
+        e
+      case other =>
+        other
+    }
+
+    private def lookupV2Function(
+        catalog: FunctionCatalog,
+        ident: Identifier,
+        arguments: Seq[Expression],
+        isDistinct: Boolean,
+        filter: Option[Expression],
+        ignoreNulls: Boolean): Expression = {
+      val unbound = catalog.loadFunction(ident)
+      val inputType = StructType(arguments.zipWithIndex.map {
+        case (exp, pos) => StructField(s"_$pos", exp.dataType, exp.nullable)
+      })
+      val bound = try {
+        unbound.bind(inputType)
+      } catch {
+        case unsupported: UnsupportedOperationException =>
+          failAnalysis(s"Function '${unbound.name}' cannot process input: " +
+            s"(${arguments.map(_.dataType.simpleString).mkString(", ")}): " +
+            unsupported.getMessage)
+      }
+
+      bound match {
+        case scalarFunc: ScalarFunction[_] =>
+          if (isDistinct) {
+            throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+              scalarFunc.name(), "DISTINCT")
+          } else if (filter.isDefined) {
+            throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+              scalarFunc.name(), "FILTER clause")
+          } else if (ignoreNulls) {
+            throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+              scalarFunc.name(), "IGNORE NULLS")
+          } else {
+            // TODO: implement type coercion by looking at input type from the UDF. We may
+            //  also want to check if the parameter types from the magic method match the
+            //  input type through `BoundFunction.inputTypes`.
+            val argClasses = inputType.fields.map(_.dataType)
+            findMethod(scalarFunc, MAGIC_METHOD_NAME, Some(argClasses)) match {
+              case Some(_) =>
+                val caller = Literal.create(scalarFunc, ObjectType(scalarFunc.getClass))
+                Invoke(caller, MAGIC_METHOD_NAME, scalarFunc.resultType(),
+                  arguments, returnNullable = scalarFunc.isResultNullable)
+              case _ =>
+                // TODO: handle functions defined in Scala too - in Scala, even if a
+                //  subclass do not override the default method in parent interface
+                //  defined in Java, the method can still be found from
+                //  `getDeclaredMethod`.
+                findMethod(scalarFunc, "produceResult", Some(Seq(inputType))) match {

Review comment:
       Looks like this works to find the `InternalRow` method because the input types are passed as a single struct type. I think that would be more clear if there were a comment explaining that's how `findMethod` is used here.




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825626040


   **[Test build #137851 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137851/testReport)** for PR 32082 at commit [`68e1001`](https://github.com/apache/spark/commit/68e10014d2793d0bbe69f93b263e9817b0fa2913).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public 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.

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-818285487


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137237/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-817994057


   **[Test build #137227 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137227/testReport)** for PR 32082 at commit [`dc9dde6`](https://github.com/apache/spark/commit/dc9dde65db5a815460e9e430acc4f07554ed0dcd).


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

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] SparkQA removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827382359


   **[Test build #137992 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137992/testReport)** for PR 32082 at commit [`c453b64`](https://github.com/apache/spark/commit/c453b64ae669c2be28a9467be8b5111a7563f2ab).


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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825520271






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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r620410980



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1955,20 +1959,29 @@ class Analyzer(override val catalogManager: CatalogManager)
    * @see https://issues.apache.org/jira/browse/SPARK-19737
    */
   object LookupFunctions extends Rule[LogicalPlan] {
+    import CatalogV2Implicits._
     override def apply(plan: LogicalPlan): LogicalPlan = {
       val externalFunctionNameSet = new mutable.HashSet[FunctionIdentifier]()
       plan.resolveExpressions {
-        case f: UnresolvedFunction
-          if externalFunctionNameSet.contains(normalizeFuncName(f.name)) => f
-        case f: UnresolvedFunction if v1SessionCatalog.isRegisteredFunction(f.name) => f
-        case f: UnresolvedFunction if v1SessionCatalog.isPersistentFunction(f.name) =>
-          externalFunctionNameSet.add(normalizeFuncName(f.name))
+        case f @ UnresolvedFunction(NonSessionCatalogAndIdentifier(_, _), _, _, _, _) =>

Review comment:
       This makes me feel that it's clearer to write if else, which also saves a lot of `AsFunctionIdentifier.unapply` calls.
   ```
   case f @ UnresolvedFunction(CatalogAndIdentifier(catalog, v2Ident), _, ...) =>
     if (CatalogV2Util.isSessionCatalog(catalog)) {
       // do nothing for v2 catalog, as we don't have api to check v2 function existence
     } else {
       val ident = v2Ident.asFunctionIdentifier
       if (externalFunctionNameSet.contains...) else if ... else ...
     }
   ```




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

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] SparkQA removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825455233


   **[Test build #137851 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137851/testReport)** for PR 32082 at commit [`68e1001`](https://github.com/apache/spark/commit/68e10014d2793d0bbe69f93b263e9817b0fa2913).


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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815478465


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137041/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825151163






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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825151835






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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827497987


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137987/
   


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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r618130754



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
##########
@@ -269,23 +269,24 @@ case class UnresolvedGenerator(name: FunctionIdentifier, children: Seq[Expressio
 }
 
 case class UnresolvedFunction(
-    name: FunctionIdentifier,
+    multipartIdentifier: Seq[String],
     arguments: Seq[Expression],
     isDistinct: Boolean,
     filter: Option[Expression] = None,
     ignoreNulls: Boolean = false)
   extends Expression with Unevaluable {
+  import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
 
   override def children: Seq[Expression] = arguments ++ filter.toSeq
 
   override def dataType: DataType = throw new UnresolvedException("dataType")
   override def nullable: Boolean = throw new UnresolvedException("nullable")
   override lazy val resolved = false
 
-  override def prettyName: String = name.unquotedString
+  override def prettyName: String = multipartIdentifier.quoted
   override def toString: String = {
     val distinct = if (isDistinct) "distinct " else ""
-    s"'$name($distinct${children.mkString(", ")})"
+    s"'${multipartIdentifier.quoted}($distinct${children.mkString(", ")})"

Review comment:
       Hmm sorry what do you mean? to be honest I'm not sure why there is a single quote here originally.




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

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] rdblue commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617870366



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
##########
@@ -296,11 +297,22 @@ case class UnresolvedFunction(
       copy(arguments = newChildren)
     }
   }
+
+  def name: FunctionIdentifier = multipartIdentifier.asFunctionIdentifier

Review comment:
       Where is this used? Since `asFunctionIdentifier` will throw exceptions for multi-part identifiers, it seems like this is dangerous. If name is used in normal places, those could suddenly start throwing errors.




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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r618105900



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala
##########
@@ -154,6 +154,26 @@ private[sql] trait LookupCatalog extends Logging {
     }
   }
 
+  object AsFunctionIdentifier {
+    def unapply(parts: Seq[String]): Option[FunctionIdentifier] = {
+      def namesToFunctionIdentifier(names: Seq[String]): Option[FunctionIdentifier] = names match {
+        case Seq(name) => Some(FunctionIdentifier(name))
+        case Seq(database, name) => Some(FunctionIdentifier(name, Some(database)))
+        case _ => None
+      }
+      parts match {
+        case CatalogAndMultipartIdentifier(None, names)
+          if CatalogV2Util.isSessionCatalog(currentCatalog) =>
+          namesToFunctionIdentifier(names)
+        case CatalogAndMultipartIdentifier(Some(catalog), names)
+          if CatalogV2Util.isSessionCatalog(catalog) &&
+            CatalogV2Util.isSessionCatalog(currentCatalog) =>

Review comment:
       I followed `AsTableIdentifier` for this check and see related comments [here](https://github.com/apache/spark/pull/30919#discussion_r549610277). But now looking at it, I think it should be OK to remove this check for both function and table identifier.




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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r620410980



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1955,20 +1959,29 @@ class Analyzer(override val catalogManager: CatalogManager)
    * @see https://issues.apache.org/jira/browse/SPARK-19737
    */
   object LookupFunctions extends Rule[LogicalPlan] {
+    import CatalogV2Implicits._
     override def apply(plan: LogicalPlan): LogicalPlan = {
       val externalFunctionNameSet = new mutable.HashSet[FunctionIdentifier]()
       plan.resolveExpressions {
-        case f: UnresolvedFunction
-          if externalFunctionNameSet.contains(normalizeFuncName(f.name)) => f
-        case f: UnresolvedFunction if v1SessionCatalog.isRegisteredFunction(f.name) => f
-        case f: UnresolvedFunction if v1SessionCatalog.isPersistentFunction(f.name) =>
-          externalFunctionNameSet.add(normalizeFuncName(f.name))
+        case f @ UnresolvedFunction(NonSessionCatalogAndIdentifier(_, _), _, _, _, _) =>

Review comment:
       This makes me feel that it's clearer to write if else, which also saves a lot of `AsFunctionIdentifier.unapply` calls.
   ```
   // check temp function
   case f @ UnresolvedFunction(Seq(name))
       if v1SessionCatalog.isRegisteredFunction(FunctionIdentifier(name)) => f
   
   case f @ UnresolvedFunction(CatalogAndIdentifier(catalog, v2Ident), _, ...) =>
     if (!CatalogV2Util.isSessionCatalog(catalog)) {
       // do nothing for v2 catalog, as we don't have api to check v2 function existence
     } else {
       val ident = v2Ident.asFunctionIdentifier
       if (externalFunctionNameSet.contains...) {
         // OK
       } else if ... {
         // OK
       } else if ... else ...
     }
   ```




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-822967110


   **[Test build #137656 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137656/testReport)** for PR 32082 at commit [`887fb59`](https://github.com/apache/spark/commit/887fb59bf61356624b8819a4faf40b97df939548).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public 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.

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825520271


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42381/
   


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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827426212


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137992/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815340673


   **[Test build #137036 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137036/testReport)** for PR 32082 at commit [`afe0f62`](https://github.com/apache/spark/commit/afe0f62ab9ee6556988f28d8e1688b049c60aee2).
    * This patch **fails Java style tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class ApplyFunctionExpression(`
     * `case class V2Aggregator[BUF <: java.io.Serializable, OUT](`
     * `  implicit class FunctionIdentifierHelper(ident: FunctionIdentifier) `


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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617544864



##########
File path: sql/core/src/test/java/test/org/apache/spark/sql/connector/catalog/functions/JavaAverage.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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 test.org.apache.spark.sql.connector.catalog.functions;
+
+import org.apache.spark.sql.connector.catalog.functions.AggregateFunction;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.IntegerType;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.Serializable;
+
+public class JavaAverage implements UnboundFunction {
+  @Override
+  public String name() {
+    return "iavg";
+  }
+
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length != 1) {
+      throw new UnsupportedOperationException("Expect exactly one argument");
+    }
+    if (inputType.fields()[0].dataType() instanceof IntegerType) {
+      return new JavaAverageNoImpl();
+    }
+    throw new UnsupportedOperationException("Unsupported non-integral type: " +
+        inputType.fields()[0].dataType());
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  public static class JavaAverageNoImpl implements AggregateFunction<State, Integer> {

Review comment:
       nit: `BoundJavaAverageNoImpl `




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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-823487331


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42226/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-821532329


   **[Test build #137496 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137496/testReport)** for PR 32082 at commit [`f76a5f7`](https://github.com/apache/spark/commit/f76a5f7ecd7e1a9f5eac196b362645fac604badc).


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815513127


   **[Test build #137064 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137064/testReport)** for PR 32082 at commit [`173f7e1`](https://github.com/apache/spark/commit/173f7e1ca6c1dece9a89655f8c8a8a29ff840a15).


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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617383228



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/V2Aggregator.scala
##########
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Expression, UnsafeProjection}
+import org.apache.spark.sql.connector.catalog.functions.{AggregateFunction => V2AggregateFunction}
+import org.apache.spark.sql.types.DataType
+
+case class V2Aggregator[BUF <: java.io.Serializable, OUT](
+    aggrFunc: V2AggregateFunction[BUF, OUT],
+    children: Seq[Expression],
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0) extends TypedImperativeAggregate[BUF] {
+  private[this] lazy val inputProjection = UnsafeProjection.create(children)
+
+  override def nullable: Boolean = aggrFunc.isResultNullable
+  override def dataType: DataType = aggrFunc.resultType()
+  override def createAggregationBuffer(): BUF = aggrFunc.newAggregationState()
+
+  override def update(buffer: BUF, input: InternalRow): BUF = {

Review comment:
       For `TypedImperativeAggregate`, the entire `update` function is a black box to Catalyst. The magic method can't help here as codegen doesn't apply here.
   
   We need to have deeper codegen integration in  `TypedImperativeAggregate`, which we may not able to complete before 3.2. I think it's OK, and we can always add magic method later as it won't break backward compatibility.
   
   @sunchao can you revisit the doc of v2 `AggregateFunction` and remove descriptions of the magic 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.

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r618112069



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1963,6 +1966,14 @@ class Analyzer(override val catalogManager: CatalogManager)
     override def apply(plan: LogicalPlan): LogicalPlan = {
       val externalFunctionNameSet = new mutable.HashSet[FunctionIdentifier]()
       plan.resolveExpressions {
+        case f @ UnresolvedFunction(NonSessionCatalogAndIdentifier(catalog, name), _, _, _, _) =>

Review comment:
       You mean in `CheckAnalysis`? to my understanding that is done at the end after we've run all the rules, while doing it here allows us to do some sanity checks and detect analysis errors earlier.




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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r618133703



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
##########
@@ -296,11 +297,22 @@ case class UnresolvedFunction(
       copy(arguments = newChildren)
     }
   }
+
+  def name: FunctionIdentifier = multipartIdentifier.asFunctionIdentifier

Review comment:
       This is currently used in a few places in `Analyzer`. Let me see if I can replace these usages using the multi-part identifier.




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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r614070471



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2025,9 +2030,74 @@ class Analyzer(override val catalogManager: CatalogManager)
                   name, other.getClass.getCanonicalName)
               }
             }
-          case u @ UnresolvedFunction(funcId, arguments, isDistinct, filter, ignoreNulls) =>
+          case UnresolvedFunction(NonSessionCatalogAndIdentifier(v2Catalog, ident), arguments,
+            isDistinct, filter, _) if v2Catalog.isFunctionCatalog =>
+            val unbound = v2Catalog.asFunctionCatalog.loadFunction(ident)
+
+            val inputType = StructType(arguments.zipWithIndex.map {
+              case (exp, pos) => StructField(s"_$pos", exp.dataType, exp.nullable)
+            })
+
+            val bound = try {
+              unbound.bind(inputType)
+            } catch {
+              case unsupported: UnsupportedOperationException =>
+                failAnalysis(s"Function '${unbound.name}' cannot process input: " +
+                  s"(${arguments.map(_.dataType.simpleString).mkString(", ")}): " +
+                  unsupported.getMessage)
+            }
+
+            bound match {
+              case scalarFunc: ScalarFunction[_] =>
+                if (isDistinct) {
+                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+                    scalarFunc.name(), "DISTINCT")
+                } else if (filter.isDefined) {
+                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+                    scalarFunc.name(), "FILTER clause")
+                } else {
+                  // TODO: implement type coercion by looking at input type from the UDF. We may
+                  //  also want to check if the parameter types from the magic method match the
+                  //  input type through `BoundFunction.inputTypes`.
+                  val argClasses = inputType.fields.map(_.dataType)
+                  findMethod(scalarFunc, ScalarFunction.MAGIC_METHOD_NAME, Some(argClasses)) match {
+                    case Some(_) =>
+                      val caller = Literal.create(scalarFunc, ObjectType(scalarFunc.getClass))
+                      Invoke(caller, ScalarFunction.MAGIC_METHOD_NAME, scalarFunc.resultType(),
+                        arguments, returnNullable = scalarFunc.isResultNullable)
+                    case _ =>
+                      // TODO: handle functions defined in Scala too - in Scala, even if a
+                      //  subclass do not override the default method in parent interface defined
+                      //  in Java, the method can still be found from `getDeclaredMethod`.
+                      findMethod(scalarFunc, "produceResult", Some(Seq(inputType))) match {

Review comment:
       I'm a bit confused. `produceResult` is defined in the interface. Why do we need reflection here?




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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-821554908


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42070/
   


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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617989198



##########
File path: sql/core/src/test/java/test/org/apache/spark/sql/connector/catalog/functions/JavaAverage.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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 test.org.apache.spark.sql.connector.catalog.functions;
+
+import org.apache.spark.sql.connector.catalog.functions.AggregateFunction;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.IntegerType;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.Serializable;
+
+public class JavaAverage implements UnboundFunction {

Review comment:
       Yes it's a big weird that this only has `JavaAverageNoImpl`. Let me add another implementation that can be used for positive 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.

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-821724587


   **[Test build #137496 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137496/testReport)** for PR 32082 at commit [`f76a5f7`](https://github.com/apache/spark/commit/f76a5f7ecd7e1a9f5eac196b362645fac604badc).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public 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.

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815567054


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/41642/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-821553133


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42070/
   


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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815478465


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137041/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825281701


   **[Test build #137834 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137834/testReport)** for PR 32082 at commit [`55c1f4b`](https://github.com/apache/spark/commit/55c1f4bcd72ffbd7000b2304833af21c0e560772).


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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825520272


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137848/
   


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

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] SparkQA removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-821530643


   **[Test build #137495 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137495/testReport)** for PR 32082 at commit [`f949f84`](https://github.com/apache/spark/commit/f949f84d9b07daa2009ac32b987f09584ce6c521).


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815355595


   **[Test build #137039 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137039/testReport)** for PR 32082 at commit [`1ae5520`](https://github.com/apache/spark/commit/1ae552090d91a2390eb6a8fb2d4908e264ea4771).


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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r620371604



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1569,8 +1573,8 @@ class Analyzer(override val catalogManager: CatalogManager)
           // results and confuse users if there is any null values. For count(t1.*, t2.*), it is
           // still allowed, since it's well-defined in spark.
           if (!conf.allowStarWithSingleTableIdentifierInCount &&
-              f1.name.database.isEmpty &&
-              f1.name.funcName == "count" &&
+              f1.nameParts.length == 1 &&
+              f1.nameParts.head == "count" &&

Review comment:
       is this simply `f1.nameParts == Seq("count")`?




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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r620711029



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1569,8 +1573,8 @@ class Analyzer(override val catalogManager: CatalogManager)
           // results and confuse users if there is any null values. For count(t1.*, t2.*), it is
           // still allowed, since it's well-defined in spark.
           if (!conf.allowStarWithSingleTableIdentifierInCount &&
-              f1.name.database.isEmpty &&
-              f1.name.funcName == "count" &&
+              f1.nameParts.length == 1 &&
+              f1.nameParts.head == "count" &&

Review comment:
       yeah your's is simpler - will 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.

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r620371604



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1569,8 +1573,8 @@ class Analyzer(override val catalogManager: CatalogManager)
           // results and confuse users if there is any null values. For count(t1.*, t2.*), it is
           // still allowed, since it's well-defined in spark.
           if (!conf.allowStarWithSingleTableIdentifierInCount &&
-              f1.name.database.isEmpty &&
-              f1.name.funcName == "count" &&
+              f1.nameParts.length == 1 &&
+              f1.nameParts.head == "count" &&

Review comment:
       is this simple `f1.nameParts == Seq("count")`?




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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r621218917



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2095,9 +2104,101 @@ class Analyzer(override val catalogManager: CatalogManager)
                 case other =>
                   other
               }
+          }
+
+          case u @ UnresolvedFunction(nameParts, arguments, isDistinct, filter, ignoreNulls) =>
+            withPosition(u) {
+              expandIdentifier(nameParts) match {
+                case NonSessionCatalogAndIdentifier(catalog, ident) =>
+                  if (!catalog.isFunctionCatalog) {
+                    throw new AnalysisException(s"Trying to lookup function '$ident' in " +
+                      s"catalog '${catalog.name()}', but it is not a FunctionCatalog.")
+                  }
+
+                  val unbound = catalog.asFunctionCatalog.loadFunction(ident)
+                  val inputType = StructType(arguments.zipWithIndex.map {
+                    case (exp, pos) => StructField(s"_$pos", exp.dataType, exp.nullable)
+                  })
+                  val bound = try {
+                    unbound.bind(inputType)
+                  } catch {
+                    case unsupported: UnsupportedOperationException =>
+                      throw new AnalysisException(s"Function '${unbound.name}' cannot process " +
+                        s"input: (${arguments.map(_.dataType.simpleString).mkString(", ")}): " +
+                        unsupported.getMessage, cause = Some(unsupported))
+                  }
+
+                  bound match {
+                    case scalarFunc: ScalarFunction[_] =>
+                      if (isDistinct) {

Review comment:
       nit: for readability, can we put the logic of handling ScalarFunction into a 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.

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815675132


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137064/
   


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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r636177774



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ApplyFunctionExpression.scala
##########
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction
+import org.apache.spark.sql.types.DataType
+
+case class ApplyFunctionExpression(
+    function: ScalarFunction[_],
+    children: Seq[Expression]) extends Expression with UserDefinedExpression with CodegenFallback {
+  override def nullable: Boolean = function.isResultNullable
+  override def name: String = function.name()
+  override def dataType: DataType = function.resultType()
+
+  private lazy val reusedRow = new GenericInternalRow(children.size)

Review comment:
       Yeah looks like we should. Let me see change this and run the benchmark again.




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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r620366218



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java
##########
@@ -23,17 +23,68 @@
 /**
  * Interface for a function that produces a result value for each input row.
  * <p>
- * For each input row, Spark will call a produceResult method that corresponds to the
- * {@link #inputTypes() input data types}. The expected JVM argument types must be the types used by
- * Spark's InternalRow API. If no direct method is found or when not using codegen, Spark will call
- * {@link #produceResult(InternalRow)}.
+ * To evaluate each input row, Spark will first try to lookup and use a "magic method" (described
+ * below) through Java reflection. If the method is not found, Spark will call
+ * {@link #produceResult(InternalRow)} as a fallback approach.
  * <p>
  * The JVM type of result values produced by this function must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.
+ * <p>
+ * <b>IMPORTANT</b>: the default implementation of {@link #produceResult} throws
+ * {@link UnsupportedOperationException}. Users can choose to override this method, or implement
+ * a "magic method" with name {@link #MAGIC_METHOD_NAME} which takes individual parameters
+ * instead of a {@link InternalRow}. The magic method will be loaded by Spark through Java
+ * reflection and will also provide better performance in general, due to optimizations such as
+ * codegen, removal of Java boxing, etc.
+ *
+ * For example, a scalar UDF for adding two integers can be defined as follow with the magic
+ * method approach:
+ *
+ * <pre>
+ *   public class IntegerAdd implements{@code ScalarFunction<Integer>} {
+ *     public int invoke(int left, int right) {
+ *       return left + right;
+ *     }
+ *   }
+ * </pre>
+ * In this case, since {@link #MAGIC_METHOD_NAME} is defined, Spark will use it over
+ * {@link #produceResult} to evalaute the inputs. In general Spark looks up the magic method by
+ * first converting the actual input SQL data types to their corresponding Java types following
+ * the mapping defined below, and then checking if there is a matching method from all the
+ * declared methods in the UDF class, using method name (i.e., {@link #MAGIC_METHOD_NAME}) and
+ * the Java types. If no magic method is found, Spark will falls back to use {@link #produceResult}.
+ * <p>
+ * The following are the mapping from {@link DataType SQL data type} to Java type through
+ * the magic method approach:
+ * <ul>
+ *   <li>{@link org.apache.spark.sql.types.BooleanType}: {@code boolean}</li>
+ *   <li>{@link org.apache.spark.sql.types.ByteType}: {@code byte}</li>
+ *   <li>{@link org.apache.spark.sql.types.ShortType}: {@code short}</li>
+ *   <li>{@link org.apache.spark.sql.types.IntegerType}: {@code int}</li>
+ *   <li>{@link org.apache.spark.sql.types.LongType}: {@code long}</li>
+ *   <li>{@link org.apache.spark.sql.types.FloatType}: {@code float}</li>
+ *   <li>{@link org.apache.spark.sql.types.DoubleType}: {@code double}</li>
+ *   <li>{@link org.apache.spark.sql.types.StringType}:
+ *       {@link org.apache.spark.unsafe.types.UTF8String}</li>
+ *   <li>{@link org.apache.spark.sql.types.DateType}: {@code int}</li>
+ *   <li>{@link org.apache.spark.sql.types.TimestampType}: {@code long}</li>
+ *   <li>{@link org.apache.spark.sql.types.BinaryType}: {@code byte[]}</li>
+ *   <li>{@link org.apache.spark.sql.types.DayTimeIntervalType}: {@code long}</li>
+ *   <li>{@link org.apache.spark.sql.types.YearMonthIntervalType}: {@code int}</li>
+ *   <li>{@link org.apache.spark.sql.types.DecimalType}:
+ *       {@link org.apache.spark.sql.types.Decimal}</li>
+ *   <li>{@link org.apache.spark.sql.types.StructType}: {@link InternalRow}</li>
+ *   <li>{@link org.apache.spark.sql.types.ArrayType}:
+ *       {@link org.apache.spark.sql.catalyst.util.ArrayData}</li>
+ *   <li>{@link org.apache.spark.sql.types.MapType}:
+ *       {@link org.apache.spark.sql.catalyst.util.MapData}</li>
+ *   <li>any other type: {@code Object}</li>

Review comment:
       when do we need 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.

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] SparkQA removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-821532329


   **[Test build #137496 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137496/testReport)** for PR 32082 at commit [`f76a5f7`](https://github.com/apache/spark/commit/f76a5f7ecd7e1a9f5eac196b362645fac604badc).


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827372255


   **[Test build #137989 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137989/testReport)** for PR 32082 at commit [`97c29c3`](https://github.com/apache/spark/commit/97c29c3aef6dbcd8fd43d4f96bcd0238b932fc02).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public 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.

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r614243784



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1967,6 +1969,9 @@ class Analyzer(override val catalogManager: CatalogManager)
     override def apply(plan: LogicalPlan): LogicalPlan = {
       val externalFunctionNameSet = new mutable.HashSet[FunctionIdentifier]()
       plan.resolveExpressions {
+        case f @ UnresolvedFunction(NonSessionCatalogAndIdentifier(_, _), _, _, _, _) =>
+          // no-op if this is from a v2 catalog

Review comment:
       Yes looking at [SPARK-19737](https://issues.apache.org/jira/browse/SPARK-19737) I think this will be very useful (although the checking won't be complete since `bind` could still fail). Do you mind if I add a TODO here and do it separately 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.

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] SparkQA removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827350141


   **[Test build #137987 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137987/testReport)** for PR 32082 at commit [`f25b5e6`](https://github.com/apache/spark/commit/f25b5e683f796c353296adb519b4882e37fbc213).


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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r618670253



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2021,88 +2032,218 @@ class Analyzer(override val catalogManager: CatalogManager)
                   name, other.getClass.getCanonicalName)
               }
             }
-          case u @ UnresolvedFunction(funcId, arguments, isDistinct, filter, ignoreNulls) =>
+
+          case u @ UnresolvedFunction(AsFunctionIdentifier(ident), arguments,
+          isDistinct, filter, ignoreNulls) =>
             withPosition(u) {
-              v1SessionCatalog.lookupFunction(funcId, arguments) match {
-                // AggregateWindowFunctions are AggregateFunctions that can only be evaluated within
-                // the context of a Window clause. They do not need to be wrapped in an
-                // AggregateExpression.
-                case wf: AggregateWindowFunction =>
-                  if (isDistinct) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      wf.prettyName, "DISTINCT")
-                  } else if (filter.isDefined) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      wf.prettyName, "FILTER clause")
-                  } else if (ignoreNulls) {
-                    wf match {
-                      case nthValue: NthValue =>
-                        nthValue.copy(ignoreNulls = ignoreNulls)
-                      case _ =>
-                        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                          wf.prettyName, "IGNORE NULLS")
-                    }
-                  } else {
-                    wf
-                  }
-                case owf: FrameLessOffsetWindowFunction =>
-                  if (isDistinct) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      owf.prettyName, "DISTINCT")
-                  } else if (filter.isDefined) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      owf.prettyName, "FILTER clause")
-                  } else if (ignoreNulls) {
-                    owf match {
-                      case lead: Lead =>
-                        lead.copy(ignoreNulls = ignoreNulls)
-                      case lag: Lag =>
-                        lag.copy(ignoreNulls = ignoreNulls)
-                    }
-                  } else {
-                    owf
-                  }
-                // We get an aggregate function, we need to wrap it in an AggregateExpression.
-                case agg: AggregateFunction =>
-                  if (filter.isDefined && !filter.get.deterministic) {
-                    throw QueryCompilationErrors.nonDeterministicFilterInAggregateError
-                  }
-                  if (ignoreNulls) {
-                    val aggFunc = agg match {
-                      case first: First => first.copy(ignoreNulls = ignoreNulls)
-                      case last: Last => last.copy(ignoreNulls = ignoreNulls)
-                      case _ =>
-                        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                          agg.prettyName, "IGNORE NULLS")
-                    }
-                    AggregateExpression(aggFunc, Complete, isDistinct, filter)
-                  } else {
-                    AggregateExpression(agg, Complete, isDistinct, filter)
-                  }
-                // This function is not an aggregate function, just return the resolved one.
-                case other if isDistinct =>
-                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                    other.prettyName, "DISTINCT")
-                case other if filter.isDefined =>
-                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                    other.prettyName, "FILTER clause")
-                case other if ignoreNulls =>
-                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                    other.prettyName, "IGNORE NULLS")
-                case e: String2TrimExpression if arguments.size == 2 =>
-                  if (trimWarningEnabled.get) {
-                    log.warn("Two-parameter TRIM/LTRIM/RTRIM function signatures are deprecated." +
-                      " Use SQL syntax `TRIM((BOTH | LEADING | TRAILING)? trimStr FROM str)`" +
-                      " instead.")
-                    trimWarningEnabled.set(false)
-                  }
-                  e
-                case other =>
-                  other
+              processFunctionExpr(v1SessionCatalog.lookupFunction(ident, arguments),
+                arguments, isDistinct, filter, ignoreNulls)
+            }
+
+          case u @ UnresolvedFunction(parts, arguments, isDistinct, filter, ignoreNulls) =>
+            withPosition(u) {
+              // resolve built-in or temporary functions with v2 catalog
+              val resultExpression = if (parts.length == 1) {
+                v1SessionCatalog.lookupBuiltinOrTempFunction(parts.head, arguments).map(
+                  processFunctionExpr(_, arguments, isDistinct, filter, ignoreNulls)
+                )
+              } else {
+                None
               }
+
+              resultExpression.getOrElse(
+                expandIdentifier(parts) match {
+                  case NonSessionCatalogAndIdentifier(catalog: FunctionCatalog, ident) =>
+                    lookupV2Function(catalog, ident, arguments, isDistinct, filter, ignoreNulls)
+                  case _ => u
+                }
+              )
             }
         }
     }
+
+    /**
+     * Check if the input `fn` implements the given `methodName`. If `inputType` is set, it also
+     * tries to match it against the declared parameter types.
+     */
+    private def findMethod(
+        fn: BoundFunction,
+        methodName: String,
+        inputTypeOpt: Option[Seq[DataType]] = None): Option[Method] = {
+      val cls = fn.getClass
+      inputTypeOpt match {
+        case Some(inputType) =>
+          try {
+            val argClasses = inputType.map(ScalaReflection.dataTypeJavaClass)
+            Some(cls.getDeclaredMethod(methodName, argClasses: _*))
+          } catch {
+            case _: NoSuchMethodException =>
+              None
+          }
+        case None =>
+          cls.getDeclaredMethods.find(_.getName == methodName)
+      }
+    }
+
+    private def processFunctionExpr(
+        expr: Expression,
+        arguments: Seq[Expression],
+        isDistinct: Boolean,
+        filter: Option[Expression],
+        ignoreNulls: Boolean): Expression = expr match {
+      // AggregateWindowFunctions are AggregateFunctions that can only be evaluated within
+      // the context of a Window clause. They do not need to be wrapped in an
+      // AggregateExpression.
+      case wf: AggregateWindowFunction =>
+        if (isDistinct) {
+          throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+            wf.prettyName, "DISTINCT")
+        } else if (filter.isDefined) {
+          throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+            wf.prettyName, "FILTER clause")
+        } else if (ignoreNulls) {
+          wf match {
+            case nthValue: NthValue =>
+              nthValue.copy(ignoreNulls = ignoreNulls)
+            case _ =>
+              throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+                wf.prettyName, "IGNORE NULLS")
+          }
+        } else {
+          wf
+        }
+      case owf: FrameLessOffsetWindowFunction =>
+        if (isDistinct) {
+          throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+            owf.prettyName, "DISTINCT")
+        } else if (filter.isDefined) {
+          throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+            owf.prettyName, "FILTER clause")
+        } else if (ignoreNulls) {
+          owf match {
+            case lead: Lead =>
+              lead.copy(ignoreNulls = ignoreNulls)
+            case lag: Lag =>
+              lag.copy(ignoreNulls = ignoreNulls)
+          }
+        } else {
+          owf
+        }
+      // We get an aggregate function, we need to wrap it in an AggregateExpression.
+      case agg: AggregateFunction =>
+        if (filter.isDefined && !filter.get.deterministic) {
+          throw QueryCompilationErrors.nonDeterministicFilterInAggregateError
+        }
+        if (ignoreNulls) {
+          val aggFunc = agg match {
+            case first: First => first.copy(ignoreNulls = ignoreNulls)
+            case last: Last => last.copy(ignoreNulls = ignoreNulls)
+            case _ =>
+              throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+                agg.prettyName, "IGNORE NULLS")
+          }
+          AggregateExpression(aggFunc, Complete, isDistinct, filter)
+        } else {
+          AggregateExpression(agg, Complete, isDistinct, filter)
+        }
+      // This function is not an aggregate function, just return the resolved one.
+      case other if isDistinct =>
+        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+          other.prettyName, "DISTINCT")
+      case other if filter.isDefined =>
+        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+          other.prettyName, "FILTER clause")
+      case other if ignoreNulls =>
+        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+          other.prettyName, "IGNORE NULLS")
+      case e: String2TrimExpression if arguments.size == 2 =>
+        if (trimWarningEnabled.get) {
+          log.warn("Two-parameter TRIM/LTRIM/RTRIM function signatures are deprecated." +
+            " Use SQL syntax `TRIM((BOTH | LEADING | TRAILING)? trimStr FROM str)`" +
+            " instead.")
+          trimWarningEnabled.set(false)
+        }
+        e
+      case other =>
+        other
+    }
+
+    private def lookupV2Function(
+        catalog: FunctionCatalog,
+        ident: Identifier,
+        arguments: Seq[Expression],
+        isDistinct: Boolean,
+        filter: Option[Expression],
+        ignoreNulls: Boolean): Expression = {
+      val unbound = catalog.loadFunction(ident)
+      val inputType = StructType(arguments.zipWithIndex.map {
+        case (exp, pos) => StructField(s"_$pos", exp.dataType, exp.nullable)
+      })
+      val bound = try {
+        unbound.bind(inputType)
+      } catch {
+        case unsupported: UnsupportedOperationException =>
+          failAnalysis(s"Function '${unbound.name}' cannot process input: " +
+            s"(${arguments.map(_.dataType.simpleString).mkString(", ")}): " +
+            unsupported.getMessage)
+      }
+
+      bound match {
+        case scalarFunc: ScalarFunction[_] =>
+          if (isDistinct) {
+            throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+              scalarFunc.name(), "DISTINCT")
+          } else if (filter.isDefined) {
+            throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+              scalarFunc.name(), "FILTER clause")
+          } else if (ignoreNulls) {
+            throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
+              scalarFunc.name(), "IGNORE NULLS")
+          } else {
+            // TODO: implement type coercion by looking at input type from the UDF. We may
+            //  also want to check if the parameter types from the magic method match the
+            //  input type through `BoundFunction.inputTypes`.
+            val argClasses = inputType.fields.map(_.dataType)
+            findMethod(scalarFunc, MAGIC_METHOD_NAME, Some(argClasses)) match {
+              case Some(_) =>
+                val caller = Literal.create(scalarFunc, ObjectType(scalarFunc.getClass))
+                Invoke(caller, MAGIC_METHOD_NAME, scalarFunc.resultType(),
+                  arguments, returnNullable = scalarFunc.isResultNullable)
+              case _ =>
+                // TODO: handle functions defined in Scala too - in Scala, even if a
+                //  subclass do not override the default method in parent interface
+                //  defined in Java, the method can still be found from
+                //  `getDeclaredMethod`.
+                findMethod(scalarFunc, "produceResult", Some(Seq(inputType))) match {

Review comment:
       Good point. Added some comments 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.

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825325261


   **[Test build #137834 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137834/testReport)** for PR 32082 at commit [`55c1f4b`](https://github.com/apache/spark/commit/55c1f4bcd72ffbd7000b2304833af21c0e560772).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public 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.

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-828037327


   **[Test build #138006 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138006/testReport)** for PR 32082 at commit [`c18715f`](https://github.com/apache/spark/commit/c18715fcc304f4a805d420e193d4a3884b3f7e4f).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public 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.

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-818191606


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/41817/
   


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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617553954



##########
File path: sql/core/src/test/java/test/org/apache/spark/sql/connector/catalog/functions/JavaAverage.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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 test.org.apache.spark.sql.connector.catalog.functions;
+
+import org.apache.spark.sql.connector.catalog.functions.AggregateFunction;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.IntegerType;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.Serializable;
+
+public class JavaAverage implements UnboundFunction {

Review comment:
       or we can follow `JavaStrLen` and allow to set different `BoundFunction` in the `UnboundFunction` implementation. 




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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617363788



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -897,9 +900,9 @@ class Analyzer(override val catalogManager: CatalogManager)
     }
   }
 
-  // If we are resolving relations insides views, we need to expand single-part relation names with
-  // the current catalog and namespace of when the view was created.
-  private def expandRelationName(nameParts: Seq[String]): Seq[String] = {
+  // If we are resolving relations insides views, we may need to expand single or multi-part

Review comment:
       `relations` -> `database objects(relation, function, etc.)`




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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r618659003



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala
##########
@@ -154,6 +154,26 @@ private[sql] trait LookupCatalog extends Logging {
     }
   }
 
+  object AsFunctionIdentifier {
+    def unapply(parts: Seq[String]): Option[FunctionIdentifier] = {
+      def namesToFunctionIdentifier(names: Seq[String]): Option[FunctionIdentifier] = names match {
+        case Seq(name) => Some(FunctionIdentifier(name))
+        case Seq(database, name) => Some(FunctionIdentifier(name, Some(database)))
+        case _ => None
+      }
+      parts match {
+        case CatalogAndMultipartIdentifier(None, names)
+          if CatalogV2Util.isSessionCatalog(currentCatalog) =>

Review comment:
       I think yes otherwise it will exceed the 100 character limit.




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-818191474






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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-822892047


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42185/
   


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

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] sunchao commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-823452368


   @cloud-fan I've addressed all the comments. Would you mind to give another round of review? cc @viirya @dongjoon-hyun too.


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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r620753098



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/higherOrderFunctions.scala
##########
@@ -30,13 +30,14 @@ import org.apache.spark.sql.types.DataType
  * so we need to resolve higher order function when all children are either resolved or a lambda
  * function.
  */
-case class ResolveHigherOrderFunctions(catalog: SessionCatalog) extends Rule[LogicalPlan] {
+case class ResolveHigherOrderFunctions(catalogManager: CatalogManager)
+  extends Rule[LogicalPlan] with LookupCatalog {
 
   override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveExpressions {
-    case u @ UnresolvedFunction(fn, children, false, filter, ignoreNulls)
+    case u @ UnresolvedFunction(AsFunctionIdentifier(ident), children, false, filter, ignoreNulls)

Review comment:
       @cloud-fan Thanks. Yes you are right, it will fail in this case. I think we can perhaps get rid of `AsFunctionIdentifier` and following similar logic as you pointed above:
   ```scala
   case u @ UnresolvedFunction(nameParts, children, false, filter, ignoreNulls)
       if hasLambdaAndResolvedArguments(children) =>
     val CatalogAndIdentifier(catalog, ident) = nameParts
     if (nameParts.length == 1 || CatalogV2Util.isSessionCatalog(catalog)) {
      ...
     }
   }
   ```
   
   




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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-817999593


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137227/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-823449486


   **[Test build #137699 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137699/testReport)** for PR 32082 at commit [`b08c814`](https://github.com/apache/spark/commit/b08c814f51daca6fb34fe862143fffe4a6e3a3d1).


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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r622367803



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java
##########
@@ -23,17 +23,67 @@
 /**
  * Interface for a function that produces a result value for each input row.
  * <p>
- * For each input row, Spark will call a produceResult method that corresponds to the
- * {@link #inputTypes() input data types}. The expected JVM argument types must be the types used by
- * Spark's InternalRow API. If no direct method is found or when not using codegen, Spark will call
- * {@link #produceResult(InternalRow)}.
+ * To evaluate each input row, Spark will first try to lookup and use a "magic method" (described
+ * below) through Java reflection. If the method is not found, Spark will call
+ * {@link #produceResult(InternalRow)} as a fallback approach.
  * <p>
  * The JVM type of result values produced by this function must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.
+ * <p>
+ * <b>IMPORTANT</b>: the default implementation of {@link #produceResult} throws
+ * {@link UnsupportedOperationException}. Users can choose to override this method, or implement
+ * a "magic method" with name {@link #MAGIC_METHOD_NAME} which takes individual parameters
+ * instead of a {@link InternalRow}. The magic method will be loaded by Spark through Java
+ * reflection and will also provide better performance in general, due to optimizations such as
+ * codegen, removal of Java boxing, etc.
+ *
+ * For example, a scalar UDF for adding two integers can be defined as follow with the magic
+ * method approach:
+ *
+ * <pre>
+ *   public class IntegerAdd implements{@code ScalarFunction<Integer>} {
+ *     public int invoke(int left, int right) {

Review comment:
       Sure will do. It should similar to the current `invoke` and we can leverage `StaticInvoke` for the purpose. Do you think we can do this in a separate PR?




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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617544482



##########
File path: sql/core/src/test/java/test/org/apache/spark/sql/connector/catalog/functions/JavaAverage.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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 test.org.apache.spark.sql.connector.catalog.functions;
+
+import org.apache.spark.sql.connector.catalog.functions.AggregateFunction;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.IntegerType;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.Serializable;
+
+public class JavaAverage implements UnboundFunction {

Review comment:
       nit: `JavaAverageNoImpl` to indicate it's for negative test




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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825444942


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42378/
   


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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r621211590



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1958,18 +1961,23 @@ class Analyzer(override val catalogManager: CatalogManager)
     override def apply(plan: LogicalPlan): LogicalPlan = {
       val externalFunctionNameSet = new mutable.HashSet[FunctionIdentifier]()
       plan.resolveExpressions {
-        case f: UnresolvedFunction
-          if externalFunctionNameSet.contains(normalizeFuncName(f.name)) => f
-        case f: UnresolvedFunction if v1SessionCatalog.isRegisteredFunction(f.name) => f
-        case f: UnresolvedFunction if v1SessionCatalog.isPersistentFunction(f.name) =>
-          externalFunctionNameSet.add(normalizeFuncName(f.name))
-          f
-        case f: UnresolvedFunction =>
-          withPosition(f) {
-            throw new NoSuchFunctionException(
-              f.name.database.getOrElse(v1SessionCatalog.getCurrentDatabase),
-              f.name.funcName)
+        case f @ UnresolvedFunction(AsFunctionIdentifier(ident), _, _, _, _) =>
+          if (externalFunctionNameSet.contains(normalizeFuncName(ident)) ||
+            v1SessionCatalog.isRegisteredFunction(ident)) {
+            f
+          } else if (v1SessionCatalog.isPersistentFunction(ident)) {
+            externalFunctionNameSet.add(normalizeFuncName(ident))
+            f
+          } else {
+            withPosition(f) {
+              throw new NoSuchFunctionException(
+                ident.database.getOrElse(v1SessionCatalog.getCurrentDatabase),
+                ident.funcName)
+            }
           }
+        case f: UnresolvedFunction =>

Review comment:
       We are inside `resolveExpressions`. We can remove this case if we do nothing.




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825441064






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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r614265700



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2025,9 +2030,74 @@ class Analyzer(override val catalogManager: CatalogManager)
                   name, other.getClass.getCanonicalName)
               }
             }
-          case u @ UnresolvedFunction(funcId, arguments, isDistinct, filter, ignoreNulls) =>
+          case UnresolvedFunction(NonSessionCatalogAndIdentifier(v2Catalog, ident), arguments,
+            isDistinct, filter, _) if v2Catalog.isFunctionCatalog =>

Review comment:
       This is a good point. We can check this in `LookupFunctions` and I think reporting that `cat` doesn't support function would be more informative in this case.




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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-821554908


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42070/
   


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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-821725854


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137496/
   


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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827381008






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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815336154


   **[Test build #137036 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137036/testReport)** for PR 32082 at commit [`afe0f62`](https://github.com/apache/spark/commit/afe0f62ab9ee6556988f28d8e1688b049c60aee2).


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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r636160964



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ApplyFunctionExpression.scala
##########
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction
+import org.apache.spark.sql.types.DataType
+
+case class ApplyFunctionExpression(
+    function: ScalarFunction[_],
+    children: Seq[Expression]) extends Expression with UserDefinedExpression with CodegenFallback {
+  override def nullable: Boolean = function.isResultNullable
+  override def name: String = function.name()
+  override def dataType: DataType = function.resultType()
+
+  private lazy val reusedRow = new GenericInternalRow(children.size)

Review comment:
       shall we use `SpecificInternalRow` to avoid boxing?




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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-823487331


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42226/
   


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

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] rdblue commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617875737



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala
##########
@@ -154,6 +154,26 @@ private[sql] trait LookupCatalog extends Logging {
     }
   }
 
+  object AsFunctionIdentifier {
+    def unapply(parts: Seq[String]): Option[FunctionIdentifier] = {
+      def namesToFunctionIdentifier(names: Seq[String]): Option[FunctionIdentifier] = names match {
+        case Seq(name) => Some(FunctionIdentifier(name))
+        case Seq(database, name) => Some(FunctionIdentifier(name, Some(database)))
+        case _ => None
+      }
+      parts match {
+        case CatalogAndMultipartIdentifier(None, names)
+          if CatalogV2Util.isSessionCatalog(currentCatalog) =>

Review comment:
       Does this need to wrap to the next line?




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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825313308


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42364/
   


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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r618669775



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala
##########
@@ -29,7 +29,7 @@ import org.scalatest.matchers.must.Matchers
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFormat, CatalogTable, CatalogTableType, ExternalCatalog, InMemoryCatalog, SessionCatalog}
 import org.apache.spark.sql.catalyst.dsl.plans._
-import org.apache.spark.sql.connector.{InMemoryTable, InMemoryTableCatalog}
+import org.apache.spark.sql.connector.{InMemoryTable, V2InMemoryCatalog}

Review comment:
       Good point. I removed most of the changes, but still kept the `InMemoryCatalog` though, thinking that it would be useful to have a test catalog with both tables and functions (similar to the V1 `InMemoryCatalog`). It is currently only used in `DataSourceV2FunctionSuite` and `CatalogSuite`.




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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825335510


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137834/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825148278


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42351/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825455233


   **[Test build #137851 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137851/testReport)** for PR 32082 at commit [`68e1001`](https://github.com/apache/spark/commit/68e10014d2793d0bbe69f93b263e9817b0fa2913).


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

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] SparkQA removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-818079333


   **[Test build #137237 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137237/testReport)** for PR 32082 at commit [`c94e7d5`](https://github.com/apache/spark/commit/c94e7d56efdef972b3f92769dfd2da3f7782c5b8).


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

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 #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-828635127


   thanks, merging to master!


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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827422935


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42512/
   


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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825151836






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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825335510


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137834/
   


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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r614063763



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java
##########
@@ -30,10 +30,71 @@
  * <p>
  * The JVM type of result values produced by this function must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.
+ * <p>
+ * <b>IMPORTANT</b>: the default implementation of {@link #produceResult} throws
+ * {@link UnsupportedOperationException}. Users can choose to override this method, or implement
+ * a "magic method" with name {@link #MAGIC_METHOD_NAME} which takes individual parameters
+ * instead of a {@link InternalRow}. The magic method will be loaded by Spark through Java
+ * reflection and also will provide better performance in general, due to optimizations such as
+ * codegen, Java boxing and so on.
+ *
+ * For example, a scalar UDF for adding two integers can be defined as follow with the magic
+ * method approach:
+ *
+ * <pre>
+ *   public class IntegerAdd implements{@code ScalarFunction<Integer>} {
+ *     public int invoke(int left, int right) {
+ *       return left + right;
+ *     }
+ *
+ *    {@literal @}Override
+ *     public produceResult(InternalRow input) {
+ *       int left = input.getInt(0);
+ *       int right = input.getInt(1);
+ *       return left + right;
+ *     }
+ *   }
+ * </pre>
+ * In this case, both {@link #MAGIC_METHOD_NAME} and {@link #produceResult} are defined, and Spark
+ * will first lookup the {@link #MAGIC_METHOD_NAME} method during query analysis. This is done by
+ * first converting the actual input SQL data types to their corresponding Java types following the
+ * mapping defined below, and then checking if there is a matching method from all the declared
+ * methods in the UDF class, using method name (i.e., {@link #MAGIC_METHOD_NAME}) and the Java
+ * types. If no magic method is found, Spark will falls back to use {@link #produceResult}.
+ * <p>
+ * The following are the mapping from {@link DataType SQL data type} to Java type through
+ * the magic method approach:
+ * <ul>
+ *   <li>{@link org.apache.spark.sql.types.BooleanType}: {@code boolean}</li>
+ *   <li>{@link org.apache.spark.sql.types.ByteType}: {@code byte}</li>
+ *   <li>{@link org.apache.spark.sql.types.ShortType}: {@code short}</li>
+ *   <li>{@link org.apache.spark.sql.types.IntegerType}: {@code int}</li>
+ *   <li>{@link org.apache.spark.sql.types.LongType}: {@code long}</li>
+ *   <li>{@link org.apache.spark.sql.types.FloatType}: {@code float}</li>
+ *   <li>{@link org.apache.spark.sql.types.DoubleType}: {@code double}</li>
+ *   <li>{@link org.apache.spark.sql.types.StringType}:
+ *       {@link org.apache.spark.unsafe.types.UTF8String}</li>
+ *   <li>{@link org.apache.spark.sql.types.DateType}: {@code int}</li>
+ *   <li>{@link org.apache.spark.sql.types.TimestampType}: {@code long}</li>
+ *   <li>{@link org.apache.spark.sql.types.BinaryType}: {@code byte[]}</li>
+ *   <li>{@link org.apache.spark.sql.types.CalendarIntervalType}:

Review comment:
       This is deprecated. Shall we exclude it from the doc?




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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r621538744



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2095,9 +2104,101 @@ class Analyzer(override val catalogManager: CatalogManager)
                 case other =>
                   other
               }
+          }
+
+          case u @ UnresolvedFunction(nameParts, arguments, isDistinct, filter, ignoreNulls) =>
+            withPosition(u) {
+              expandIdentifier(nameParts) match {
+                case NonSessionCatalogAndIdentifier(catalog, ident) =>
+                  if (!catalog.isFunctionCatalog) {
+                    throw new AnalysisException(s"Trying to lookup function '$ident' in " +
+                      s"catalog '${catalog.name()}', but it is not a FunctionCatalog.")
+                  }
+
+                  val unbound = catalog.asFunctionCatalog.loadFunction(ident)
+                  val inputType = StructType(arguments.zipWithIndex.map {
+                    case (exp, pos) => StructField(s"_$pos", exp.dataType, exp.nullable)
+                  })
+                  val bound = try {
+                    unbound.bind(inputType)
+                  } catch {
+                    case unsupported: UnsupportedOperationException =>
+                      throw new AnalysisException(s"Function '${unbound.name}' cannot process " +
+                        s"input: (${arguments.map(_.dataType.simpleString).mkString(", ")}): " +
+                        unsupported.getMessage, cause = Some(unsupported))
+                  }
+
+                  bound match {
+                    case scalarFunc: ScalarFunction[_] =>
+                      if (isDistinct) {

Review comment:
       Sure, done. I think we can also consider moving the big chunk of handling of V1 and V2 functions separately into two functions like what I used to have:
   ```scala
   case u @ UnresolvedFunction(AsFunctionIdentifier(ident), arguments, isDistinct, 
     filter, ignoreNulls) => withPosition(u) {
       processV1Function(...)
     }
   
   case u @ UnresolvedFunction(nameParts, arguments, isDistinct, filter, ignoreNulls) =>
     withPosition(u) {
       processV2Function(...)
     }
   ```
   to keep the main logic of pattern matching on unresolved functions clearer. Let me know if you prefer this way too.




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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827908675


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42525/
   


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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r615601637



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/V2Aggregator.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.connector.catalog.functions.{AggregateFunction => V2AggregateFunction}
+import org.apache.spark.sql.types.DataType
+
+case class V2Aggregator[BUF <: java.io.Serializable, OUT](
+    aggrFunc: V2AggregateFunction[BUF, OUT],
+    children: Seq[Expression],
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0) extends TypedImperativeAggregate[BUF] {
+  override def createAggregationBuffer(): BUF = aggrFunc.newAggregationState()
+
+  override def update(buffer: BUF, input: InternalRow): BUF = aggrFunc.update(buffer, input)

Review comment:
       SGTM, let's wrap `children.map(_.eval(input))` into an `InternalRow` and more forward 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.

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815388876






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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827606682


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137994/
   


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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r615599252



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java
##########
@@ -30,10 +30,71 @@
  * <p>
  * The JVM type of result values produced by this function must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.
+ * <p>
+ * <b>IMPORTANT</b>: the default implementation of {@link #produceResult} throws
+ * {@link UnsupportedOperationException}. Users can choose to override this method, or implement
+ * a "magic method" with name {@link #MAGIC_METHOD_NAME} which takes individual parameters

Review comment:
       let's update it to match the actual behavior.




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825148035


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42352/
   


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

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 closed pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #32082:
URL: https://github.com/apache/spark/pull/32082


   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827355087


   **[Test build #137989 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137989/testReport)** for PR 32082 at commit [`97c29c3`](https://github.com/apache/spark/commit/97c29c3aef6dbcd8fd43d4f96bcd0238b932fc02).


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

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] github-actions[bot] commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-821530899


   **[Test build #756770086](https://github.com/sunchao/spark/actions/runs/756770086)** for PR 32082 at commit [`f76a5f7`](https://github.com/sunchao/spark/commit/f76a5f7ecd7e1a9f5eac196b362645fac604badc).


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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r621530817



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1958,18 +1961,23 @@ class Analyzer(override val catalogManager: CatalogManager)
     override def apply(plan: LogicalPlan): LogicalPlan = {
       val externalFunctionNameSet = new mutable.HashSet[FunctionIdentifier]()
       plan.resolveExpressions {
-        case f: UnresolvedFunction
-          if externalFunctionNameSet.contains(normalizeFuncName(f.name)) => f
-        case f: UnresolvedFunction if v1SessionCatalog.isRegisteredFunction(f.name) => f
-        case f: UnresolvedFunction if v1SessionCatalog.isPersistentFunction(f.name) =>
-          externalFunctionNameSet.add(normalizeFuncName(f.name))
-          f
-        case f: UnresolvedFunction =>
-          withPosition(f) {
-            throw new NoSuchFunctionException(
-              f.name.database.getOrElse(v1SessionCatalog.getCurrentDatabase),
-              f.name.funcName)
+        case f @ UnresolvedFunction(AsFunctionIdentifier(ident), _, _, _, _) =>
+          if (externalFunctionNameSet.contains(normalizeFuncName(ident)) ||
+            v1SessionCatalog.isRegisteredFunction(ident)) {
+            f
+          } else if (v1SessionCatalog.isPersistentFunction(ident)) {
+            externalFunctionNameSet.add(normalizeFuncName(ident))
+            f
+          } else {
+            withPosition(f) {
+              throw new NoSuchFunctionException(
+                ident.database.getOrElse(v1SessionCatalog.getCurrentDatabase),
+                ident.funcName)
+            }
           }
+        case f: UnresolvedFunction =>

Review comment:
       Oops didn't realize that. Removed for now but we may add back in future when we have V2 API for checking function existence.




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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r618192035



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
##########
@@ -269,23 +269,24 @@ case class UnresolvedGenerator(name: FunctionIdentifier, children: Seq[Expressio
 }
 
 case class UnresolvedFunction(
-    name: FunctionIdentifier,
+    multipartIdentifier: Seq[String],
     arguments: Seq[Expression],
     isDistinct: Boolean,
     filter: Option[Expression] = None,
     ignoreNulls: Boolean = false)
   extends Expression with Unevaluable {
+  import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
 
   override def children: Seq[Expression] = arguments ++ filter.toSeq
 
   override def dataType: DataType = throw new UnresolvedException("dataType")
   override def nullable: Boolean = throw new UnresolvedException("nullable")
   override lazy val resolved = false
 
-  override def prettyName: String = name.unquotedString
+  override def prettyName: String = multipartIdentifier.quoted
   override def toString: String = {
     val distinct = if (isDistinct) "distinct " else ""
-    s"'$name($distinct${children.mkString(", ")})"
+    s"'${multipartIdentifier.quoted}($distinct${children.mkString(", ")})"

Review comment:
       Thanks @cloud-fan ! in that case I think it is still valid here?




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

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] SparkQA removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815375315


   **[Test build #137041 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137041/testReport)** for PR 32082 at commit [`c522276`](https://github.com/apache/spark/commit/c522276bf1f051af6200c17bdf51a4aa0f565b0a).


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827900598


   Kubernetes integration test unable to build dist.
   
   exiting with code: 1
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42525/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-821530643


   **[Test build #137495 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137495/testReport)** for PR 32082 at commit [`f949f84`](https://github.com/apache/spark/commit/f949f84d9b07daa2009ac32b987f09584ce6c521).


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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r621530003



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/AggregateFunction.java
##########
@@ -25,12 +25,9 @@
 /**
  * Interface for a function that produces a result value by aggregating over multiple input rows.
  * <p>
- * For each input row, Spark will call an update method that corresponds to the
- * {@link #inputTypes() input data types}. The expected JVM argument types must be the types used by
- * Spark's InternalRow API. If no direct method is found or when not using codegen, Spark will call
- * update with {@link InternalRow}.
- * <p>
- * The JVM type of result values produced by this function must be the type used by Spark's
+ * For each input row, Spark will call the {@link #update} method which should evaluate the row
+ * and update the aggregation state. The JVM type of result values produced by
+ * {@link #produceResult} must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.

Review comment:
       Good idea. Added.




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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r618659686



##########
File path: sql/core/src/test/java/test/org/apache/spark/sql/connector/catalog/functions/JavaAverage.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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 test.org.apache.spark.sql.connector.catalog.functions;
+
+import org.apache.spark.sql.connector.catalog.functions.AggregateFunction;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.IntegerType;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.Serializable;
+
+public class JavaAverage implements UnboundFunction {

Review comment:
       I changed this to wrap a `JavaDoubleAverage` function since we removed the default method from `AggregateFunction`.




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-818267981


   **[Test build #137237 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137237/testReport)** for PR 32082 at commit [`c94e7d5`](https://github.com/apache/spark/commit/c94e7d56efdef972b3f92769dfd2da3f7782c5b8).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public 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.

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] rdblue commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617869582



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
##########
@@ -269,23 +269,24 @@ case class UnresolvedGenerator(name: FunctionIdentifier, children: Seq[Expressio
 }
 
 case class UnresolvedFunction(
-    name: FunctionIdentifier,
+    multipartIdentifier: Seq[String],
     arguments: Seq[Expression],
     isDistinct: Boolean,
     filter: Option[Expression] = None,
     ignoreNulls: Boolean = false)
   extends Expression with Unevaluable {
+  import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
 
   override def children: Seq[Expression] = arguments ++ filter.toSeq
 
   override def dataType: DataType = throw new UnresolvedException("dataType")
   override def nullable: Boolean = throw new UnresolvedException("nullable")
   override lazy val resolved = false
 
-  override def prettyName: String = name.unquotedString
+  override def prettyName: String = multipartIdentifier.quoted
   override def toString: String = {
     val distinct = if (isDistinct) "distinct " else ""
-    s"'$name($distinct${children.mkString(", ")})"
+    s"'${multipartIdentifier.quoted}($distinct${children.mkString(", ")})"

Review comment:
       Is the single quote still valid here?




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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617952799



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/V2Aggregator.scala
##########
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Expression, UnsafeProjection}
+import org.apache.spark.sql.connector.catalog.functions.{AggregateFunction => V2AggregateFunction}
+import org.apache.spark.sql.types.DataType
+
+case class V2Aggregator[BUF <: java.io.Serializable, OUT](
+    aggrFunc: V2AggregateFunction[BUF, OUT],
+    children: Seq[Expression],
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0) extends TypedImperativeAggregate[BUF] {
+  private[this] lazy val inputProjection = UnsafeProjection.create(children)
+
+  override def nullable: Boolean = aggrFunc.isResultNullable
+  override def dataType: DataType = aggrFunc.resultType()
+  override def createAggregationBuffer(): BUF = aggrFunc.newAggregationState()
+
+  override def update(buffer: BUF, input: InternalRow): BUF = {

Review comment:
       Yes I think it may require some work to fully support codegen and magic method in aggregate function. How about we change the `AggregateFunction#update` to non-default method so users have to implement it? We can switch to default and throw exception when we have the magic method support in future and it should be backward compatible. cc @rdblue 




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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r618660141



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala
##########
@@ -0,0 +1,425 @@
+/*
+ * 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.connector
+
+import java.util
+import java.util.Collections
+
+import test.org.apache.spark.sql.connector.catalog.functions.{JavaAverage, JavaStrLen}
+import test.org.apache.spark.sql.connector.catalog.functions.JavaStrLen._
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.{AnalysisException, Row}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.connector.catalog.{Identifier, SupportsNamespaces}
+import org.apache.spark.sql.connector.catalog.functions._
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.UTF8String
+
+class DataSourceV2FunctionSuite extends DatasourceV2SQLBase {
+  private val emptyProps: util.Map[String, String] = Collections.emptyMap[String, String]
+
+  private def addFunction(ident: Identifier, fn: UnboundFunction): Unit = {
+    catalog("testcat").asInstanceOf[V2InMemoryCatalog].createFunction(ident, fn)
+  }
+
+  test("undefined function") {
+    assert(intercept[AnalysisException](
+      sql("SELECT testcat.non_exist('abc')").collect()
+    ).getMessage.contains("Undefined function"))
+  }
+
+  test("non-function catalog") {
+    spark.conf.set("spark.sql.catalog.testcat", classOf[BasicInMemoryTableCatalog].getName)

Review comment:
       Good point! fixed.




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825499852


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42381/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-818079333


   **[Test build #137237 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137237/testReport)** for PR 32082 at commit [`c94e7d5`](https://github.com/apache/spark/commit/c94e7d56efdef972b3f92769dfd2da3f7782c5b8).


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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r614238811



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java
##########
@@ -30,10 +30,71 @@
  * <p>
  * The JVM type of result values produced by this function must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.
+ * <p>
+ * <b>IMPORTANT</b>: the default implementation of {@link #produceResult} throws
+ * {@link UnsupportedOperationException}. Users can choose to override this method, or implement
+ * a "magic method" with name {@link #MAGIC_METHOD_NAME} which takes individual parameters
+ * instead of a {@link InternalRow}. The magic method will be loaded by Spark through Java
+ * reflection and also will provide better performance in general, due to optimizations such as
+ * codegen, Java boxing and so on.
+ *
+ * For example, a scalar UDF for adding two integers can be defined as follow with the magic
+ * method approach:
+ *
+ * <pre>
+ *   public class IntegerAdd implements{@code ScalarFunction<Integer>} {
+ *     public int invoke(int left, int right) {
+ *       return left + right;
+ *     }
+ *
+ *    {@literal @}Override
+ *     public produceResult(InternalRow input) {
+ *       int left = input.getInt(0);
+ *       int right = input.getInt(1);
+ *       return left + right;
+ *     }
+ *   }
+ * </pre>
+ * In this case, both {@link #MAGIC_METHOD_NAME} and {@link #produceResult} are defined, and Spark
+ * will first lookup the {@link #MAGIC_METHOD_NAME} method during query analysis. This is done by
+ * first converting the actual input SQL data types to their corresponding Java types following the
+ * mapping defined below, and then checking if there is a matching method from all the declared
+ * methods in the UDF class, using method name (i.e., {@link #MAGIC_METHOD_NAME}) and the Java
+ * types. If no magic method is found, Spark will falls back to use {@link #produceResult}.
+ * <p>
+ * The following are the mapping from {@link DataType SQL data type} to Java type through
+ * the magic method approach:
+ * <ul>
+ *   <li>{@link org.apache.spark.sql.types.BooleanType}: {@code boolean}</li>
+ *   <li>{@link org.apache.spark.sql.types.ByteType}: {@code byte}</li>
+ *   <li>{@link org.apache.spark.sql.types.ShortType}: {@code short}</li>
+ *   <li>{@link org.apache.spark.sql.types.IntegerType}: {@code int}</li>
+ *   <li>{@link org.apache.spark.sql.types.LongType}: {@code long}</li>
+ *   <li>{@link org.apache.spark.sql.types.FloatType}: {@code float}</li>
+ *   <li>{@link org.apache.spark.sql.types.DoubleType}: {@code double}</li>
+ *   <li>{@link org.apache.spark.sql.types.StringType}:
+ *       {@link org.apache.spark.unsafe.types.UTF8String}</li>
+ *   <li>{@link org.apache.spark.sql.types.DateType}: {@code int}</li>
+ *   <li>{@link org.apache.spark.sql.types.TimestampType}: {@code long}</li>
+ *   <li>{@link org.apache.spark.sql.types.BinaryType}: {@code byte[]}</li>
+ *   <li>{@link org.apache.spark.sql.types.CalendarIntervalType}:

Review comment:
       Sure. Where I can see more info of this? seems [the class](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala) is still marked as stable and there is no info about the deprecation. 




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815371805






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

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] SparkQA removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815336154


   **[Test build #137036 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137036/testReport)** for PR 32082 at commit [`afe0f62`](https://github.com/apache/spark/commit/afe0f62ab9ee6556988f28d8e1688b049c60aee2).


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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r620444812



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/higherOrderFunctions.scala
##########
@@ -30,13 +30,14 @@ import org.apache.spark.sql.types.DataType
  * so we need to resolve higher order function when all children are either resolved or a lambda
  * function.
  */
-case class ResolveHigherOrderFunctions(catalog: SessionCatalog) extends Rule[LogicalPlan] {
+case class ResolveHigherOrderFunctions(catalogManager: CatalogManager)
+  extends Rule[LogicalPlan] with LookupCatalog {
 
   override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveExpressions {
-    case u @ UnresolvedFunction(fn, children, false, filter, ignoreNulls)
+    case u @ UnresolvedFunction(AsFunctionIdentifier(ident), children, false, filter, ignoreNulls)

Review comment:
       If the current catalog is a custom v2 catalog, and the function name has only a single part, seems `AsFunctionIdentifier` always return None, while we should treat it as a temp function 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.

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827422624


   **[Test build #137994 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137994/testReport)** for PR 32082 at commit [`790d27f`](https://github.com/apache/spark/commit/790d27f900c0ed456c198a5175410961223990f8).


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-818001595


   Kubernetes integration test unable to build dist.
   
   exiting with code: 1
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/41807/
   


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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827461262


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42515/
   


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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r614231563



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java
##########
@@ -30,10 +30,71 @@
  * <p>
  * The JVM type of result values produced by this function must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.
+ * <p>
+ * <b>IMPORTANT</b>: the default implementation of {@link #produceResult} throws
+ * {@link UnsupportedOperationException}. Users can choose to override this method, or implement
+ * a "magic method" with name {@link #MAGIC_METHOD_NAME} which takes individual parameters
+ * instead of a {@link InternalRow}. The magic method will be loaded by Spark through Java
+ * reflection and also will provide better performance in general, due to optimizations such as
+ * codegen, Java boxing and so on.
+ *
+ * For example, a scalar UDF for adding two integers can be defined as follow with the magic
+ * method approach:
+ *
+ * <pre>
+ *   public class IntegerAdd implements{@code ScalarFunction<Integer>} {
+ *     public int invoke(int left, int right) {
+ *       return left + right;
+ *     }
+ *
+ *    {@literal @}Override
+ *     public produceResult(InternalRow input) {

Review comment:
       Yes. I think I'll just remove it from the example then as it might cause some confusion.




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

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] SparkQA removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-822867284


   **[Test build #137656 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137656/testReport)** for PR 32082 at commit [`887fb59`](https://github.com/apache/spark/commit/887fb59bf61356624b8819a4faf40b97df939548).


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825114810


   **[Test build #137820 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137820/testReport)** for PR 32082 at commit [`66dac00`](https://github.com/apache/spark/commit/66dac00f7d417452dc7eb268475b22ddcbb40075).


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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-828050729


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/138006/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825496443


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42381/
   


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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827422935


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42512/
   


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

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] SparkQA removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-825119137


   **[Test build #137822 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137822/testReport)** for PR 32082 at commit [`c31a70b`](https://github.com/apache/spark/commit/c31a70bed54e00005a5fd2bbd4d5f4da9a17ec4e).


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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r614067666



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2107,6 +2177,29 @@ class Analyzer(override val catalogManager: CatalogManager)
             }
         }
     }
+
+    /**
+     * Check if the input `fn` implements the given `methodName`. If `inputType` is set, it also
+     * tries to match it against the declared parameter types.
+     */
+    private def findMethod(
+        fn: BoundFunction,
+        methodName: String,
+        inputTypeOpt: Option[Seq[DataType]] = None): Option[Method] = {
+      val cls = fn.getClass
+      inputTypeOpt match {
+        case Some(inputType) =>
+          try {
+            val argClasses = inputType.map(ScalaReflection.dataTypeJavaClass)
+            Some(cls.getDeclaredMethod(methodName, argClasses: _*))
+          } catch {
+            case _: NoSuchMethodException =>
+              None
+          }
+        case None =>
+          cls.getDeclaredMethods.find(_.getName == methodName)

Review comment:
       shall we check if it's a 0-arg 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.

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-818001611


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/41807/
   


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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r614075574



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2025,9 +2030,74 @@ class Analyzer(override val catalogManager: CatalogManager)
                   name, other.getClass.getCanonicalName)
               }
             }
-          case u @ UnresolvedFunction(funcId, arguments, isDistinct, filter, ignoreNulls) =>
+          case UnresolvedFunction(NonSessionCatalogAndIdentifier(v2Catalog, ident), arguments,
+            isDistinct, filter, _) if v2Catalog.isFunctionCatalog =>

Review comment:
       If the query refers to a function from catalog `cat`, but `cat` is not a `FunctionCatalog`, what error we should throw? `cat` doesn't support function or function not exist?




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827863630


   **[Test build #138006 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138006/testReport)** for PR 32082 at commit [`c18715f`](https://github.com/apache/spark/commit/c18715fcc304f4a805d420e193d4a3884b3f7e4f).


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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827381010






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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r618116254



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2021,88 +2032,218 @@ class Analyzer(override val catalogManager: CatalogManager)
                   name, other.getClass.getCanonicalName)
               }
             }
-          case u @ UnresolvedFunction(funcId, arguments, isDistinct, filter, ignoreNulls) =>
+
+          case u @ UnresolvedFunction(AsFunctionIdentifier(ident), arguments,
+          isDistinct, filter, ignoreNulls) =>
             withPosition(u) {
-              v1SessionCatalog.lookupFunction(funcId, arguments) match {
-                // AggregateWindowFunctions are AggregateFunctions that can only be evaluated within
-                // the context of a Window clause. They do not need to be wrapped in an
-                // AggregateExpression.
-                case wf: AggregateWindowFunction =>
-                  if (isDistinct) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      wf.prettyName, "DISTINCT")
-                  } else if (filter.isDefined) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      wf.prettyName, "FILTER clause")
-                  } else if (ignoreNulls) {
-                    wf match {
-                      case nthValue: NthValue =>
-                        nthValue.copy(ignoreNulls = ignoreNulls)
-                      case _ =>
-                        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                          wf.prettyName, "IGNORE NULLS")
-                    }
-                  } else {
-                    wf
-                  }
-                case owf: FrameLessOffsetWindowFunction =>
-                  if (isDistinct) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      owf.prettyName, "DISTINCT")
-                  } else if (filter.isDefined) {
-                    throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                      owf.prettyName, "FILTER clause")
-                  } else if (ignoreNulls) {
-                    owf match {
-                      case lead: Lead =>
-                        lead.copy(ignoreNulls = ignoreNulls)
-                      case lag: Lag =>
-                        lag.copy(ignoreNulls = ignoreNulls)
-                    }
-                  } else {
-                    owf
-                  }
-                // We get an aggregate function, we need to wrap it in an AggregateExpression.
-                case agg: AggregateFunction =>
-                  if (filter.isDefined && !filter.get.deterministic) {
-                    throw QueryCompilationErrors.nonDeterministicFilterInAggregateError
-                  }
-                  if (ignoreNulls) {
-                    val aggFunc = agg match {
-                      case first: First => first.copy(ignoreNulls = ignoreNulls)
-                      case last: Last => last.copy(ignoreNulls = ignoreNulls)
-                      case _ =>
-                        throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                          agg.prettyName, "IGNORE NULLS")
-                    }
-                    AggregateExpression(aggFunc, Complete, isDistinct, filter)
-                  } else {
-                    AggregateExpression(agg, Complete, isDistinct, filter)
-                  }
-                // This function is not an aggregate function, just return the resolved one.
-                case other if isDistinct =>
-                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                    other.prettyName, "DISTINCT")
-                case other if filter.isDefined =>
-                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                    other.prettyName, "FILTER clause")
-                case other if ignoreNulls =>
-                  throw QueryCompilationErrors.functionWithUnsupportedSyntaxError(
-                    other.prettyName, "IGNORE NULLS")
-                case e: String2TrimExpression if arguments.size == 2 =>
-                  if (trimWarningEnabled.get) {
-                    log.warn("Two-parameter TRIM/LTRIM/RTRIM function signatures are deprecated." +
-                      " Use SQL syntax `TRIM((BOTH | LEADING | TRAILING)? trimStr FROM str)`" +
-                      " instead.")
-                    trimWarningEnabled.set(false)
-                  }
-                  e
-                case other =>
-                  other
+              processFunctionExpr(v1SessionCatalog.lookupFunction(ident, arguments),
+                arguments, isDistinct, filter, ignoreNulls)
+            }
+
+          case u @ UnresolvedFunction(parts, arguments, isDistinct, filter, ignoreNulls) =>
+            withPosition(u) {
+              // resolve built-in or temporary functions with v2 catalog
+              val resultExpression = if (parts.length == 1) {

Review comment:
       I tried, but there is a dependency between this and the following case where we lookup the V2 function catalog. If we make them two separate cases, I think we won't be able to express the dependency and the control flow will always hit the first match.




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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815567054


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/41642/
   


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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r615067938



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/V2Aggregator.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.connector.catalog.functions.{AggregateFunction => V2AggregateFunction}
+import org.apache.spark.sql.types.DataType
+
+case class V2Aggregator[BUF <: java.io.Serializable, OUT](
+    aggrFunc: V2AggregateFunction[BUF, OUT],
+    children: Seq[Expression],
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0) extends TypedImperativeAggregate[BUF] {
+  override def createAggregationBuffer(): BUF = aggrFunc.newAggregationState()
+
+  override def update(buffer: BUF, input: InternalRow): BUF = aggrFunc.update(buffer, input)

Review comment:
       @cloud-fan I checked the magic method support for aggregate functions and there are a few issues:
   1. seems we can't reuse the existing `Invoke` as the `invokeUpdate` method will also take aggregate state `BUF` as input, while `Invoke` evaluates on `InternalRow` only.
   2. only hash aggregation support codegen at the moment and also it doesn't support `ImperativeAggregate`.
   
   Do you mind we leave this as a TODO for now and handle it separately? I think it requires some work to implement 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.

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] rdblue commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617879688



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala
##########
@@ -0,0 +1,425 @@
+/*
+ * 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.connector
+
+import java.util
+import java.util.Collections
+
+import test.org.apache.spark.sql.connector.catalog.functions.{JavaAverage, JavaStrLen}
+import test.org.apache.spark.sql.connector.catalog.functions.JavaStrLen._
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.{AnalysisException, Row}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.connector.catalog.{Identifier, SupportsNamespaces}
+import org.apache.spark.sql.connector.catalog.functions._
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.UTF8String
+
+class DataSourceV2FunctionSuite extends DatasourceV2SQLBase {
+  private val emptyProps: util.Map[String, String] = Collections.emptyMap[String, String]
+
+  private def addFunction(ident: Identifier, fn: UnboundFunction): Unit = {
+    catalog("testcat").asInstanceOf[V2InMemoryCatalog].createFunction(ident, fn)
+  }
+
+  test("undefined function") {
+    assert(intercept[AnalysisException](
+      sql("SELECT testcat.non_exist('abc')").collect()
+    ).getMessage.contains("Undefined function"))
+  }
+
+  test("non-function catalog") {
+    spark.conf.set("spark.sql.catalog.testcat", classOf[BasicInMemoryTableCatalog].getName)

Review comment:
       Shouldn't these tests use `withConf` to ensure the configuration settings are removed?




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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r618670766



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
##########
@@ -296,11 +297,22 @@ case class UnresolvedFunction(
       copy(arguments = newChildren)
     }
   }
+
+  def name: FunctionIdentifier = multipartIdentifier.asFunctionIdentifier

Review comment:
       I removed this. Let me know if you have any other concern.




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-817999550


   **[Test build #137227 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137227/testReport)** for PR 32082 at commit [`dc9dde6`](https://github.com/apache/spark/commit/dc9dde65db5a815460e9e430acc4f07554ed0dcd).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public 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.

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815413210


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137039/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815369757






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

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] sunchao commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-828686966


   Thanks all!


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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r614065452



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1967,6 +1969,9 @@ class Analyzer(override val catalogManager: CatalogManager)
     override def apply(plan: LogicalPlan): LogicalPlan = {
       val externalFunctionNameSet = new mutable.HashSet[FunctionIdentifier]()
       plan.resolveExpressions {
+        case f @ UnresolvedFunction(NonSessionCatalogAndIdentifier(_, _), _, _, _, _) =>
+          // no-op if this is from a v2 catalog

Review comment:
       shall we add the v2 API to check function existence?




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

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-822973964


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137656/
   


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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827497987


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137987/
   


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

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 change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r621206480



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/AggregateFunction.java
##########
@@ -25,12 +25,9 @@
 /**
  * Interface for a function that produces a result value by aggregating over multiple input rows.
  * <p>
- * For each input row, Spark will call an update method that corresponds to the
- * {@link #inputTypes() input data types}. The expected JVM argument types must be the types used by
- * Spark's InternalRow API. If no direct method is found or when not using codegen, Spark will call
- * update with {@link InternalRow}.
- * <p>
- * The JVM type of result values produced by this function must be the type used by Spark's
+ * For each input row, Spark will call the {@link #update} method which should evaluate the row
+ * and update the aggregation state. The JVM type of result values produced by
+ * {@link #produceResult} must be the type used by Spark's
  * InternalRow API for the {@link DataType SQL data type} returned by {@link #resultType()}.

Review comment:
       shall we reference the classdoc of `ScalarFunction` for the type mapping?




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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r620896602



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1955,20 +1959,29 @@ class Analyzer(override val catalogManager: CatalogManager)
    * @see https://issues.apache.org/jira/browse/SPARK-19737
    */
   object LookupFunctions extends Rule[LogicalPlan] {
+    import CatalogV2Implicits._
     override def apply(plan: LogicalPlan): LogicalPlan = {
       val externalFunctionNameSet = new mutable.HashSet[FunctionIdentifier]()
       plan.resolveExpressions {
-        case f: UnresolvedFunction
-          if externalFunctionNameSet.contains(normalizeFuncName(f.name)) => f
-        case f: UnresolvedFunction if v1SessionCatalog.isRegisteredFunction(f.name) => f
-        case f: UnresolvedFunction if v1SessionCatalog.isPersistentFunction(f.name) =>
-          externalFunctionNameSet.add(normalizeFuncName(f.name))
+        case f @ UnresolvedFunction(NonSessionCatalogAndIdentifier(_, _), _, _, _, _) =>

Review comment:
       Good point. I've combined these clauses into one. Actually, I think it may be better to add the `v1SessionCatalog.isRegisteredFunction(FunctionIdentifier(name))` into `AsFunctionIdentifier`, so that we can just have this logic:
   ```
   case f @ UnresolvedFunction(AsFunctionIdentifier(ident), ...) =>
     // process V1 session catalog function
   
   case f @ UnresolvedFunction(nameParts, ...) =>
     // process V2 catalog function
   ```
   
   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.

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] AmplabJenkins commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-823628524


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137699/
   


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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r614262615



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2107,6 +2177,29 @@ class Analyzer(override val catalogManager: CatalogManager)
             }
         }
     }
+
+    /**
+     * Check if the input `fn` implements the given `methodName`. If `inputType` is set, it also
+     * tries to match it against the declared parameter types.
+     */
+    private def findMethod(
+        fn: BoundFunction,
+        methodName: String,
+        inputTypeOpt: Option[Seq[DataType]] = None): Option[Method] = {
+      val cls = fn.getClass
+      inputTypeOpt match {
+        case Some(inputType) =>
+          try {
+            val argClasses = inputType.map(ScalaReflection.dataTypeJavaClass)
+            Some(cls.getDeclaredMethod(methodName, argClasses: _*))
+          } catch {
+            case _: NoSuchMethodException =>
+              None
+          }
+        case None =>
+          cls.getDeclaredMethods.find(_.getName == methodName)

Review comment:
       Hmm sorry not sure what you mean. The case when arg types are known is handled above. This is for `AggregateFunction.update` where we just check if the method is overridden: due to type erasure we can't do an exact match with argument types.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -2107,6 +2177,29 @@ class Analyzer(override val catalogManager: CatalogManager)
             }
         }
     }
+
+    /**
+     * Check if the input `fn` implements the given `methodName`. If `inputType` is set, it also
+     * tries to match it against the declared parameter types.
+     */
+    private def findMethod(
+        fn: BoundFunction,
+        methodName: String,
+        inputTypeOpt: Option[Seq[DataType]] = None): Option[Method] = {
+      val cls = fn.getClass
+      inputTypeOpt match {
+        case Some(inputType) =>
+          try {
+            val argClasses = inputType.map(ScalaReflection.dataTypeJavaClass)
+            Some(cls.getDeclaredMethod(methodName, argClasses: _*))
+          } catch {
+            case _: NoSuchMethodException =>
+              None
+          }
+        case None =>
+          cls.getDeclaredMethods.find(_.getName == methodName)

Review comment:
       Hmm sorry not sure what you mean. The case when arg types are known is handled above (including the 0-arg case). This is for `AggregateFunction.update` where we just check if the method is overridden: due to type erasure we can't do an exact match with argument types.




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

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] AmplabJenkins removed a comment on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827384320


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42509/
   


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827425712


   **[Test build #137992 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137992/testReport)** for PR 32082 at commit [`c453b64`](https://github.com/apache/spark/commit/c453b64ae669c2be28a9467be8b5111a7563f2ab).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public 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.

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815375315


   **[Test build #137041 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137041/testReport)** for PR 32082 at commit [`c522276`](https://github.com/apache/spark/commit/c522276bf1f051af6200c17bdf51a4aa0f565b0a).


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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827587767


   **[Test build #137994 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137994/testReport)** for PR 32082 at commit [`790d27f`](https://github.com/apache/spark/commit/790d27f900c0ed456c198a5175410961223990f8).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public 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.

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] rdblue commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r617869311



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/higherOrderFunctions.scala
##########
@@ -33,10 +34,10 @@ import org.apache.spark.sql.types.DataType
 case class ResolveHigherOrderFunctions(catalog: SessionCatalog) extends Rule[LogicalPlan] {
 
   override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveExpressions {
-    case u @ UnresolvedFunction(fn, children, false, filter, ignoreNulls)
+    case u @ UnresolvedFunction(parts, children, false, filter, ignoreNulls)
         if hasLambdaAndResolvedArguments(children) =>
       withPosition(u) {
-        catalog.lookupFunction(fn, children) match {
+        catalog.lookupFunction(parts.asFunctionIdentifier, children) match {

Review comment:
       `asFunctionIdentifier` could fail and throw an exception for some identifiers. I think this should avoid that case by limiting the matches for `parts` in the case above.




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-827457463






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

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] sunchao commented on a change in pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #32082:
URL: https://github.com/apache/spark/pull/32082#discussion_r618105900



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala
##########
@@ -154,6 +154,26 @@ private[sql] trait LookupCatalog extends Logging {
     }
   }
 
+  object AsFunctionIdentifier {
+    def unapply(parts: Seq[String]): Option[FunctionIdentifier] = {
+      def namesToFunctionIdentifier(names: Seq[String]): Option[FunctionIdentifier] = names match {
+        case Seq(name) => Some(FunctionIdentifier(name))
+        case Seq(database, name) => Some(FunctionIdentifier(name, Some(database)))
+        case _ => None
+      }
+      parts match {
+        case CatalogAndMultipartIdentifier(None, names)
+          if CatalogV2Util.isSessionCatalog(currentCatalog) =>
+          namesToFunctionIdentifier(names)
+        case CatalogAndMultipartIdentifier(Some(catalog), names)
+          if CatalogV2Util.isSessionCatalog(catalog) &&
+            CatalogV2Util.isSessionCatalog(currentCatalog) =>

Review comment:
       I followed `AsTableIdentifier` for this check and see related comments [here](https://github.com/apache/spark/pull/30919#discussion_r549610277). But looking at it now, I think it should be OK to remove this check for both function and table identifier.




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

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] SparkQA commented on pull request #32082: [SPARK-34981][SQL] Implement V2 function resolution and evaluation

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32082:
URL: https://github.com/apache/spark/pull/32082#issuecomment-815409002


   **[Test build #137039 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137039/testReport)** for PR 32082 at commit [`1ae5520`](https://github.com/apache/spark/commit/1ae552090d91a2390eb6a8fb2d4908e264ea4771).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public 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.

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