ibzib commented on a change in pull request #12169:
URL: https://github.com/apache/beam/pull/12169#discussion_r450436253



##########
File path: 
sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLDialectSpecTest.java
##########
@@ -2916,6 +2917,57 @@ public void testRecursiveUdfThrowsException() {
     zetaSQLQueryPlanner.convertToBeamRel(sql);
   }
 
+  @Test
+  public void testUDTVF() {
+    String sql =
+        "CREATE TABLE FUNCTION CustomerRange(MinID INT64, MaxID INT64)\n"
+            + "  AS\n"
+            + "    SELECT *\n"
+            + "    FROM KeyValue\n"
+            + "    WHERE key >= MinId AND key <= MaxId; \n"
+            + " SELECT key FROM CustomerRange(10, 14)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, 
beamRelNode);
+
+    Schema singleField = Schema.builder().addInt64Field("field1").build();
+    
PAssert.that(stream).containsInAnyOrder(Row.withSchema(singleField).addValues(14L).build());
+    
pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testUDTVFTableNotFound() {
+    String sql =
+        "CREATE TABLE FUNCTION CustomerRange(MinID INT64, MaxID INT64)\n"
+            + "  AS\n"
+            + "    SELECT *\n"
+            + "    FROM TableNotExist\n"
+            + "    WHERE key >= MinId AND key <= MaxId; \n"
+            + " SELECT key FROM CustomerRange(10, 14)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    thrown.expect(SqlConversionException.class);
+    thrown.expectMessage("Wasn't able to find resolve the path [TableNotExist] 
in beam");

Review comment:
       A couple nits about this error message:
   - Remove "find."
   - Instead of just "in beam" it should be "in *schema* beam" so it's clear 
what the name is referring to.

##########
File path: 
sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TVFScanConverter.java
##########
@@ -42,45 +42,57 @@
   @Override
   public RelNode convert(ResolvedTVFScan zetaNode, List<RelNode> inputs) {
     RelNode input = inputs.get(0);
+    RexCall call =
+        getExpressionConverter()
+            .convertTableValuedFunction(
+                input,
+                zetaNode.getTvf(),
+                zetaNode.getArgumentList(),
+                zetaNode.getArgumentList().get(0).getScan() != null
+                    ? 
zetaNode.getArgumentList().get(0).getScan().getColumnList()
+                    : Collections.emptyList());
     RelNode tableFunctionScan =
         LogicalTableFunctionScan.create(
-            getCluster(),
-            inputs,
-            getExpressionConverter()
-                .convertTableValuedFunction(
-                    input,
-                    zetaNode.getTvf(),
-                    zetaNode.getArgumentList(),
-                    
zetaNode.getArgumentList().get(0).getScan().getColumnList()),
-            null,
-            createRowTypeWithWindowStartAndEnd(input.getRowType()),
-            Collections.EMPTY_SET);
+            getCluster(), inputs, call, null, call.getType(), 
Collections.EMPTY_SET);
 
+    // Pure SQL UDF's language body is built bottom up, so 
FunctionArgumentRefMapping should be
+    // already consumed thus it can be cleared now.
+    context.clearFunctionArgumentRefMapping();

Review comment:
       Why do we need to clear?

##########
File path: 
sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlAnalyzer.java
##########
@@ -177,12 +179,17 @@ static AnalyzerOptions initAnalyzerOptions() {
                     LanguageFeature.FEATURE_DISALLOW_GROUP_BY_FLOAT,
                     LanguageFeature.FEATURE_V_1_2_CIVIL_TIME,
                     LanguageFeature.FEATURE_V_1_1_SELECT_STAR_EXCEPT_REPLACE,
-                    LanguageFeature.FEATURE_TABLE_VALUED_FUNCTIONS)));
+                    LanguageFeature.FEATURE_TABLE_VALUED_FUNCTIONS,
+                    LanguageFeature.FEATURE_CREATE_TABLE_FUNCTION,
+                    LanguageFeature.FEATURE_TEMPLATE_FUNCTIONS)));
 
     options
         .getLanguageOptions()
         .setSupportedStatementKinds(
-            ImmutableSet.of(RESOLVED_QUERY_STMT, 
RESOLVED_CREATE_FUNCTION_STMT));
+            ImmutableSet.of(

Review comment:
       We should make this a constant so we can reference it above.

##########
File path: 
sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlAnalyzer.java
##########
@@ -156,7 +157,8 @@ ResolvedStatement analyzeNextStatement(
                 "Failed to define function %s", String.join(".", 
createFunctionStmt.getNamePath())),
             e);
       }
-    } else if (resolvedStatement.nodeKind() != RESOLVED_QUERY_STMT) {
+    } else if (resolvedStatement.nodeKind() != RESOLVED_QUERY_STMT

Review comment:
       We should check set membership instead to make this more 
readable/scalable.

##########
File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ZetaSqlUserDefinedPureSqlTableValuedFunction.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.impl;
+
+import java.util.List;
+import org.apache.beam.sdk.annotations.Internal;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlOperandTypeInference;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlReturnTypeInference;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.validate.SqlUserDefinedFunction;
+
+/** This is a class to indicate that a TVF is a ZetaSQL pure SQL UDTVF. */
+@Internal
+public class ZetaSqlUserDefinedPureSqlTableValuedFunction extends 
SqlUserDefinedFunction {

Review comment:
       Nit: `PureSql` is probably not necessary, since we already have 
`ZetaSql` in the class name.

##########
File path: 
sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ConversionContext.java
##########
@@ -27,24 +33,39 @@
   private final ExpressionConverter expressionConverter;
   private final RelOptCluster cluster;
   private final QueryTrait trait;
+  private final Map<List<String>, ResolvedNode> sqlUDTVF;

Review comment:
       Please avoid abbreviation here. I recommend 
`userDefinedTableValuedFunctions` (`sql` is probably redundant in this case).
   
   Also consider adding an explanatory comment.

##########
File path: 
sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ConversionContext.java
##########
@@ -62,4 +83,20 @@ RelOptCluster cluster() {
   QueryTrait getTrait() {
     return trait;
   }
+
+  Map<List<String>, ResolvedNode> getSqlUDTVF() {
+    return sqlUDTVF;
+  }
+
+  Map<String, RexNode> getFunctionArgumentRefMapping() {
+    return functionArgumentRefMapping;
+  }
+
+  void addToFunctionArgumentRefMapping(String s, RexNode r) {

Review comment:
       Optional nit: `addToFunctionArgumentRefMapping` and 
`clearFunctionArgumentRefMapping` can be replaced by 
`getFunctionArgumentRefMapping().put()` and 
`getFunctionArgumentRefMapping().clear()`.

##########
File path: 
sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java
##########
@@ -171,6 +178,18 @@ public RelRoot rel(String sql, QueryParameters params) {
                 SqlAnalyzer.USER_DEFINED_FUNCTIONS,
                 String.join(".", createFunctionStmt.getNamePath()));
         udfBuilder.put(functionFullName, createFunctionStmt);
+      } else if (statement.nodeKind() == RESOLVED_CREATE_TABLE_FUNCTION_STMT) {
+        ResolvedCreateTableFunctionStmt createTableFunctionStmt =
+            (ResolvedCreateTableFunctionStmt) statement;
+        catalog.addTableValuedFunction(

Review comment:
       We add UDF to the catalog in SqlAnalyzer. It's probably fine to add to 
the catalog in either SqlAnalyzer or ZetaSQLPlannerImpl, but please keep it 
consistent.

##########
File path: 
sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java
##########
@@ -171,6 +178,18 @@ public RelRoot rel(String sql, QueryParameters params) {
                 SqlAnalyzer.USER_DEFINED_FUNCTIONS,
                 String.join(".", createFunctionStmt.getNamePath()));

Review comment:
       I suppose I should have used getNamePath instead of getFullName...

##########
File path: 
sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/FilterScanConverter.java
##########
@@ -47,7 +46,7 @@ public RelNode convert(ResolvedFilterScan zetaNode, 
List<RelNode> inputs) {
                 zetaNode.getFilterExpr(),
                 zetaNode.getInputScan().getColumnList(),
                 input.getRowType().getFieldList(),
-                ImmutableMap.of());
+                context.getFunctionArgumentRefMapping());

Review comment:
       Should we should change all converters to use 
`getFunctionArgumentRefMapping`?




----------------------------------------------------------------
This is an automated message from the 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:
us...@infra.apache.org


Reply via email to