This is an automated email from the ASF dual-hosted git repository.
ibzib pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git
The following commit(s) were added to refs/heads/master by this push:
new 8b050a3 [BEAM-10925] Change UserFunctionDefinitions into an AutoValue
and Builder class.
new 356e2ae Merge pull request #13625 from ibzib/ufd-builder
8b050a3 is described below
commit 8b050a33cfd49b7cc33a5b2c507cbc403d12fd1b
Author: Kyle Weaver <[email protected]>
AuthorDate: Mon Dec 28 11:29:53 2020 -0800
[BEAM-10925] Change UserFunctionDefinitions into an AutoValue and Builder
class.
We are planning on adding several new types of UDF, and adding them all in
the constructor will get verbose.
---
.../extensions/sql/zetasql/ZetaSQLPlannerImpl.java | 5 +++-
.../sql/zetasql/translation/ConversionContext.java | 2 +-
.../zetasql/translation/ExpressionConverter.java | 4 +++-
.../translation/UserFunctionDefinitions.java | 28 +++++++++++++++-------
4 files changed, 27 insertions(+), 12 deletions(-)
diff --git
a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java
b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java
index 6259d12..ea9e332 100644
---
a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java
+++
b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java
@@ -131,7 +131,10 @@ class ZetaSQLPlannerImpl {
}
UserFunctionDefinitions userFunctionDefinitions =
- new UserFunctionDefinitions(udfBuilder.build(), udtvfBuilder.build());
+ UserFunctionDefinitions.newBuilder()
+ .setSqlScalarFunctions(udfBuilder.build())
+ .setSqlTableValuedFunctions(udtvfBuilder.build())
+ .build();
ExpressionConverter expressionConverter =
new ExpressionConverter(cluster, params, userFunctionDefinitions);
diff --git
a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ConversionContext.java
b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ConversionContext.java
index 9523b76..3f87c54 100644
---
a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ConversionContext.java
+++
b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ConversionContext.java
@@ -79,7 +79,7 @@ public class ConversionContext {
}
Map<List<String>, ResolvedNode> getUserDefinedTableValuedFunctions() {
- return
getExpressionConverter().userFunctionDefinitions.sqlTableValuedFunctions;
+ return
getExpressionConverter().userFunctionDefinitions.sqlTableValuedFunctions();
}
Map<String, RexNode> getFunctionArgumentRefMapping() {
diff --git
a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
index a4d0f03..c732c43 100644
---
a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
+++
b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
@@ -667,7 +667,9 @@ public class ExpressionConverter {
}
} else if (USER_DEFINED_FUNCTIONS.equals(funGroup)) {
ResolvedCreateFunctionStmt createFunctionStmt =
-
userFunctionDefinitions.sqlScalarFunctions.get(functionCall.getFunction().getNamePath());
+ userFunctionDefinitions
+ .sqlScalarFunctions()
+ .get(functionCall.getFunction().getNamePath());
ResolvedExpr functionExpression =
createFunctionStmt.getFunctionExpression();
ImmutableMap.Builder<String, RexNode> innerFunctionArguments =
ImmutableMap.builder();
for (int i = 0; i < functionCall.getArgumentList().size(); i++) {
diff --git
a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/UserFunctionDefinitions.java
b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/UserFunctionDefinitions.java
index a4544e7..792c31c 100644
---
a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/UserFunctionDefinitions.java
+++
b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/UserFunctionDefinitions.java
@@ -17,27 +17,37 @@
*/
package org.apache.beam.sdk.extensions.sql.zetasql.translation;
+import com.google.auto.value.AutoValue;
import com.google.zetasql.resolvedast.ResolvedNode;
import com.google.zetasql.resolvedast.ResolvedNodes;
import java.util.List;
import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
/** Holds user defined function definitions. */
-public class UserFunctionDefinitions {
- public final ImmutableMap<List<String>,
ResolvedNodes.ResolvedCreateFunctionStmt>
- sqlScalarFunctions;
+@AutoValue
+public abstract class UserFunctionDefinitions {
+ public abstract ImmutableMap<List<String>,
ResolvedNodes.ResolvedCreateFunctionStmt>
+ sqlScalarFunctions();
/**
* SQL native user-defined table-valued function can be resolved by
Analyzer. Keeping the function
* name to its ResolvedNode mapping so during Plan conversion, UDTVF
implementation can replace
* inputs of TVFScanConverter.
*/
- public final ImmutableMap<List<String>, ResolvedNode>
sqlTableValuedFunctions;
+ public abstract ImmutableMap<List<String>, ResolvedNode>
sqlTableValuedFunctions();
- public UserFunctionDefinitions(
- ImmutableMap<List<String>, ResolvedNodes.ResolvedCreateFunctionStmt>
sqlScalarFunctions,
- ImmutableMap<List<String>, ResolvedNode> sqlTableValuedFunctions) {
- this.sqlScalarFunctions = sqlScalarFunctions;
- this.sqlTableValuedFunctions = sqlTableValuedFunctions;
+ @AutoValue.Builder
+ public abstract static class Builder {
+ public abstract Builder setSqlScalarFunctions(
+ ImmutableMap<List<String>, ResolvedNodes.ResolvedCreateFunctionStmt>
sqlScalarFunctions);
+
+ public abstract Builder setSqlTableValuedFunctions(
+ ImmutableMap<List<String>, ResolvedNode> sqlTableValuedFunctions);
+
+ public abstract UserFunctionDefinitions build();
+ }
+
+ public static Builder newBuilder() {
+ return new AutoValue_UserFunctionDefinitions.Builder();
}
}