You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2020/11/11 20:28:03 UTC

[GitHub] [beam] apilloud commented on a change in pull request #13200: [BEAM-10925] Implement Java UDF in ZetaSQL.

apilloud commented on a change in pull request #13200:
URL: https://github.com/apache/beam/pull/13200#discussion_r521602604



##########
File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlAnalyzer.java
##########
@@ -116,6 +144,51 @@ static boolean isEndOfInput(ParseResumeLocation parseResumeLocation) {
     return tables.build();
   }
 
+  /** Returns the fully qualified name of the function defined in {@code statement}. */
+  static String getFunctionQualifiedName(ResolvedCreateFunctionStmt createFunctionStmt) {

Review comment:
       nit: This is unused?

##########
File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamJavaUdfCalcRule.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.beam.sdk.extensions.sql.zetasql;
+
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamCalcRel;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Convention;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Calc;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalCalc;
+
+/** {@link ConverterRule} to replace {@link Calc} with {@link BeamCalcRel}. */
+public class BeamJavaUdfCalcRule extends ConverterRule {
+  public static final BeamJavaUdfCalcRule INSTANCE = new BeamJavaUdfCalcRule();
+
+  private BeamJavaUdfCalcRule() {
+    super(
+        LogicalCalc.class, Convention.NONE, BeamLogicalConvention.INSTANCE, "BeamJavaUdfCalcRule");
+  }
+
+  @Override
+  public boolean matches(RelOptRuleCall x) {
+    return ZetaSQLQueryPlanner.hasUdfInProjects(x);

Review comment:
       This needs to reject non-udf operations in the projects, otherwise you might end up running ZetaSQL operators through `BeamCalcRel` which currently has no coverage in compliance testing and will introduce data corruption bugs. (If you want a more expansive set of operators we need to setup the compliance tests to cover them first.)

##########
File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlAnalyzer.java
##########
@@ -116,6 +144,51 @@ static boolean isEndOfInput(ParseResumeLocation parseResumeLocation) {
     return tables.build();
   }
 
+  /** Returns the fully qualified name of the function defined in {@code statement}. */
+  static String getFunctionQualifiedName(ResolvedCreateFunctionStmt createFunctionStmt) {
+    return String.format(
+        "%s:%s",
+        getFunctionGroup(createFunctionStmt), String.join(".", createFunctionStmt.getNamePath()));
+  }
+
+  static String getFunctionGroup(ResolvedCreateFunctionStmt createFunctionStmt) {
+    switch (createFunctionStmt.getLanguage().toUpperCase()) {
+      case "JAVA":
+        return createFunctionStmt.getIsAggregate()
+            ? USER_DEFINED_JAVA_AGGREGATE_FUNCTIONS
+            : USER_DEFINED_JAVA_SCALAR_FUNCTIONS;
+      case "SQL":

Review comment:
       This field being unset is `SQL` in BigQuery. Should we match?

##########
File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java
##########
@@ -97,6 +109,30 @@ public QueryPlanner createPlanner(
     return modifyRuleSetsForZetaSql(BeamRuleSets.getRuleSets());
   }
 
+  /** Returns true if the argument contains any user-defined Java functions. */
+  static boolean hasUdfInProjects(RelOptRuleCall x) {
+    List<RelNode> resList = x.getRelList();
+    for (RelNode relNode : resList) {
+      if (relNode instanceof LogicalCalc) {
+        LogicalCalc logicalCalc = (LogicalCalc) relNode;
+        for (RexNode rexNode : logicalCalc.getProgram().getExprList()) {
+          if (rexNode instanceof RexCall) {
+            RexCall call = (RexCall) rexNode;
+            if (call.getOperator() instanceof SqlUserDefinedFunction) {
+              SqlUserDefinedFunction udf = (SqlUserDefinedFunction) call.op;
+              if (udf.function instanceof ZetaSqlScalarFunctionImpl) {
+                ZetaSqlScalarFunctionImpl scalarFunction = (ZetaSqlScalarFunctionImpl) udf.function;
+                return scalarFunction.functionGroup.equals(

Review comment:
       Nit: put the constant first to reduce risk of NPE. USER_DEFINED_JAVA_SCALAR_FUNCTIONS.equals(...)

##########
File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlAnalyzer.java
##########
@@ -116,6 +144,51 @@ static boolean isEndOfInput(ParseResumeLocation parseResumeLocation) {
     return tables.build();
   }
 
+  /** Returns the fully qualified name of the function defined in {@code statement}. */
+  static String getFunctionQualifiedName(ResolvedCreateFunctionStmt createFunctionStmt) {
+    return String.format(
+        "%s:%s",
+        getFunctionGroup(createFunctionStmt), String.join(".", createFunctionStmt.getNamePath()));
+  }
+
+  static String getFunctionGroup(ResolvedCreateFunctionStmt createFunctionStmt) {
+    switch (createFunctionStmt.getLanguage().toUpperCase()) {
+      case "JAVA":
+        return createFunctionStmt.getIsAggregate()
+            ? USER_DEFINED_JAVA_AGGREGATE_FUNCTIONS
+            : USER_DEFINED_JAVA_SCALAR_FUNCTIONS;
+      case "SQL":
+        if (createFunctionStmt.getIsAggregate()) {
+          throw new UnsupportedOperationException(
+              "Native SQL aggregate functions are not supported (BEAM-9954).");
+        }
+        return USER_DEFINED_FUNCTIONS;
+      case "PY":
+      case "PYTHON":
+      case "JS":
+      case "JAVASCRIPT":

Review comment:
       nit: How does this list compare with BigQuery? Do we need to reserve these before we implement them?

##########
File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/AggregateScanConverter.java
##########
@@ -232,4 +265,8 @@ private AggregateCall convertAggCall(
     return AggregateCall.create(
         sqlAggFunction, false, false, false, argList, -1, RelCollations.EMPTY, returnType, aggName);
   }
+
+  private static RelDataTypeFactory createTypeFactory() {

Review comment:
       nit: Nothing uses this?

##########
File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/AggregateScanConverter.java
##########
@@ -198,14 +204,41 @@ private AggregateCall convertAggCall(
               + " aggregation.");
     }
 
-    SqlAggFunction sqlAggFunction =
-        (SqlAggFunction)
-            SqlOperatorMappingTable.ZETASQL_FUNCTION_TO_CALCITE_SQL_OPERATOR.get(
-                aggregateFunctionCall.getFunction().getName());
-    if (sqlAggFunction == null) {
-      throw new UnsupportedOperationException(
-          "Does not support ZetaSQL aggregate function: "
-              + aggregateFunctionCall.getFunction().getName());
+    SqlAggFunction sqlAggFunction;
+    if (aggregateFunctionCall

Review comment:
       nit: USER_DEFINED_JAVA_AGGREGATE_FUNCTIONS.equals(...)

##########
File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
##########
@@ -644,7 +649,7 @@ private RexNode convertResolvedFunctionCall(
           throw new UnsupportedOperationException(
               "Unsupported function: " + funName + ". Only support TUMBLE, HOP, and SESSION now.");
       }
-    } else if (funGroup.equals("ZetaSQL")) {
+    } else if (funGroup.equals(ZETASQL_FUNCTION_GROUP_NAME)) {

Review comment:
       nit: ZETASQL_FUNCTION_GROUP_NAME.equals(funGroup) (There are a bunch more of this pattern, I'm going to stop commenting on them.)

##########
File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java
##########
@@ -98,22 +102,36 @@ public RelRoot rel(String sql, QueryParameters params) {
     SimpleCatalog catalog =
         analyzer.createPopulatedCatalog(defaultSchemaPlus.getName(), options, tables);
 
-    ImmutableMap.Builder<String, ResolvedCreateFunctionStmt> udfBuilder = ImmutableMap.builder();
+    ImmutableMap.Builder<List<String>, ResolvedCreateFunctionStmt> udfBuilder =
+        ImmutableMap.builder();
     ImmutableMap.Builder<List<String>, ResolvedNode> udtvfBuilder = ImmutableMap.builder();
+    ImmutableMap.Builder<List<String>, ScalarFn> javaScalarFunctionBuilder = ImmutableMap.builder();
+    ImmutableMap.Builder<List<String>, AggregateFn> javaAggregateFunctionBuilder =
+        ImmutableMap.builder();
+    JavaUdfLoader javaUdfLoader = new JavaUdfLoader();
 
     ResolvedStatement statement;
     ParseResumeLocation parseResumeLocation = new ParseResumeLocation(sql);
     do {
       statement = analyzer.analyzeNextStatement(parseResumeLocation, options, catalog);
       if (statement.nodeKind() == RESOLVED_CREATE_FUNCTION_STMT) {
         ResolvedCreateFunctionStmt createFunctionStmt = (ResolvedCreateFunctionStmt) statement;
-        // ResolvedCreateFunctionStmt does not include the full function name, so build it here.
-        String functionFullName =
-            String.format(
-                "%s:%s",
-                SqlAnalyzer.USER_DEFINED_FUNCTIONS,
-                String.join(".", createFunctionStmt.getNamePath()));
-        udfBuilder.put(functionFullName, createFunctionStmt);
+        if (SqlAnalyzer.getFunctionGroup(createFunctionStmt)

Review comment:
       nit: Could this be a switch statement 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