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/05/19 00:01:17 UTC

[GitHub] [beam] apilloud commented on a change in pull request #10946: [BEAM-9363] TUMBLE as TVF

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



##########
File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TVFScanConverter.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.translation;
+
+import com.google.zetasql.resolvedast.ResolvedNode;
+import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedTVFArgument;
+import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedTVFScan;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+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.logical.LogicalTableFunctionScan;
+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.rel.type.RelDataTypeField;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelRecordType;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
+
+class TVFScanConverter extends RelConverter<ResolvedTVFScan> {
+
+  TVFScanConverter(ConversionContext context) {
+    super(context);
+  }
+
+  @Override
+  public RelNode convert(ResolvedTVFScan zetaNode, List<RelNode> inputs) {
+    RelNode input = inputs.get(0);
+    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);
+
+    return tableFunctionScan;
+  }
+
+  @Override
+  public List<ResolvedNode> getInputs(ResolvedTVFScan zetaNode) {
+    List<ResolvedNode> inputs = new ArrayList();
+    for (ResolvedTVFArgument argument : zetaNode.getArgumentList()) {
+      if (argument.getScan() != null) {
+        inputs.add(argument.getScan());
+      }
+    }
+    return inputs;
+  }
+
+  private RelDataType createRowTypeWithWindowStartAndEnd(RelDataType inputRowType) {
+    List<RelDataTypeField> newFields = new ArrayList<>(inputRowType.getFieldList());
+    RelDataType timestampType = getCluster().getTypeFactory().createSqlType(SqlTypeName.TIMESTAMP);
+
+    RelDataTypeField windowStartField =
+        new RelDataTypeFieldImpl("window_start", newFields.size(), timestampType);

Review comment:
       Use the constant?

##########
File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java
##########
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.sql.impl.rel;
+
+import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+
+import java.lang.reflect.Type;
+import java.util.List;
+import java.util.Set;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel;
+import org.apache.beam.sdk.extensions.sql.impl.planner.NodeStats;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
+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.core.TableFunctionScan;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelColumnMapping;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+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.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.joda.time.Duration;
+
+/**
+ * BeamRelNode to replace {@code TableFunctionScan}. Currently this class limits to support
+ * table-valued function for streaming windowing.
+ */
+public class BeamTableFunctionScanRel extends TableFunctionScan implements BeamRelNode {
+  public BeamTableFunctionScanRel(
+      RelOptCluster cluster,
+      RelTraitSet traitSet,
+      List<RelNode> inputs,
+      RexNode rexCall,
+      Type elementType,
+      RelDataType rowType,
+      Set<RelColumnMapping> columnMappings) {
+    super(cluster, traitSet, inputs, rexCall, elementType, rowType, columnMappings);
+  }
+
+  @Override
+  public TableFunctionScan copy(
+      RelTraitSet traitSet,
+      List<RelNode> list,
+      RexNode rexNode,
+      Type type,
+      RelDataType relDataType,
+      Set<RelColumnMapping> set) {
+    return new BeamTableFunctionScanRel(
+        getCluster(), traitSet, list, rexNode, type, relDataType, columnMappings);

Review comment:
       Shouldn't `columnMappings` here be `set`? This is discarding the value provided as an argument to copy.

##########
File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TVFScanConverter.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.translation;
+
+import com.google.zetasql.resolvedast.ResolvedNode;
+import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedTVFArgument;
+import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedTVFScan;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+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.logical.LogicalTableFunctionScan;
+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.rel.type.RelDataTypeField;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelRecordType;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
+
+/** Converts TVFScan. */
+class TVFScanConverter extends RelConverter<ResolvedTVFScan> {
+
+  TVFScanConverter(ConversionContext context) {
+    super(context);
+  }
+
+  @Override
+  public RelNode convert(ResolvedTVFScan zetaNode, List<RelNode> inputs) {
+    RelNode input = inputs.get(0);
+    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);
+
+    return tableFunctionScan;
+  }
+
+  @Override
+  public List<ResolvedNode> getInputs(ResolvedTVFScan zetaNode) {
+    List<ResolvedNode> inputs = new ArrayList();
+    for (ResolvedTVFArgument argument : zetaNode.getArgumentList()) {
+      if (argument.getScan() != null) {
+        inputs.add(argument.getScan());
+      }
+    }
+    return inputs;
+  }
+
+  private RelDataType createRowTypeWithWindowStartAndEnd(RelDataType inputRowType) {
+    List<RelDataTypeField> newFields = new ArrayList<>(inputRowType.getFieldList());
+    RelDataType timestampType = getCluster().getTypeFactory().createSqlType(SqlTypeName.TIMESTAMP);
+
+    RelDataTypeField windowStartField =
+        new RelDataTypeFieldImpl("window_start", newFields.size(), timestampType);
+    newFields.add(windowStartField);
+    RelDataTypeField windowEndField =
+        new RelDataTypeFieldImpl("window_end", newFields.size(), timestampType);

Review comment:
       Use the constant?

##########
File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlAnalyzer.java
##########
@@ -175,6 +185,37 @@ private void addBuiltinFunctionsToCatalog(SimpleCatalog catalog, AnalyzerOptions
                     Mode.SCALAR,
                     ImmutableList.of(resolvedFunc.getSignature())))
         .forEach(catalog::addFunction);
+
+    FunctionArgumentType retType =
+        new FunctionArgumentType(SignatureArgumentKind.ARG_TYPE_RELATION);
+
+    FunctionArgumentType inputTableType =
+        new FunctionArgumentType(SignatureArgumentKind.ARG_TYPE_RELATION);
+
+    FunctionArgumentType descriptorType =
+        new FunctionArgumentType(
+            SignatureArgumentKind.ARG_TYPE_DESCRIPTOR,
+            FunctionArgumentTypeOptionsProto.newBuilder()
+                .setDescriptorResolutionTableOffset(0)
+                .build(),
+            1);
+
+    FunctionArgumentType stringType =
+        new FunctionArgumentType(TypeFactory.createSimpleType(TypeKind.TYPE_STRING));
+
+    // TUMBLE
+    catalog.addTableValuedFunction(
+        new TableValuedFunction.ForwardInputSchemaToOutputSchemaWithAppendedColumnTVF(

Review comment:
       I really hope you got a badge for that function name...

##########
File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java
##########
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.sql.impl.rel;
+
+import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+
+import java.lang.reflect.Type;
+import java.util.List;
+import java.util.Set;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel;
+import org.apache.beam.sdk.extensions.sql.impl.planner.NodeStats;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
+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.core.TableFunctionScan;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelColumnMapping;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+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.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.joda.time.Duration;
+
+/**
+ * BeamRelNode to replace {@code TableFunctionScan}. Currently this class limits to support
+ * table-valued function for streaming windowing.
+ */
+public class BeamTableFunctionScanRel extends TableFunctionScan implements BeamRelNode {
+  public BeamTableFunctionScanRel(
+      RelOptCluster cluster,
+      RelTraitSet traitSet,
+      List<RelNode> inputs,
+      RexNode rexCall,
+      Type elementType,
+      RelDataType rowType,
+      Set<RelColumnMapping> columnMappings) {
+    super(cluster, traitSet, inputs, rexCall, elementType, rowType, columnMappings);
+  }
+
+  @Override
+  public TableFunctionScan copy(
+      RelTraitSet traitSet,
+      List<RelNode> list,
+      RexNode rexNode,
+      Type type,
+      RelDataType relDataType,
+      Set<RelColumnMapping> set) {
+    return new BeamTableFunctionScanRel(
+        getCluster(), traitSet, list, rexNode, type, relDataType, columnMappings);
+  }
+
+  @Override
+  public PTransform<PCollectionList<Row>, PCollection<Row>> buildPTransform() {
+    return new Transform();
+  }
+
+  private class Transform extends PTransform<PCollectionList<Row>, PCollection<Row>> {
+
+    @Override
+    public PCollection<Row> expand(PCollectionList<Row> input) {
+      checkArgument(
+          input.size() == 1,
+          "Wrong number of inputs for %s, expected 1 input but received: %s",
+          BeamTableFunctionScanRel.class.getSimpleName(),
+          input);
+      String operatorName = ((RexCall) getCall()).getOperator().getName();
+      checkArgument(
+          operatorName.equals("TUMBLE"),
+          "Only support TUMBLE table-valued function. Current operator: %s",
+          operatorName);
+      RexCall call = ((RexCall) getCall());
+      RexInputRef wmCol = (RexInputRef) call.getOperands().get(1);
+      PCollection<Row> upstream = input.get(0);
+      Schema outputSchema = CalciteUtils.toSchema(getRowType());
+      return upstream
+          .apply(
+              ParDo.of(
+                  new FixedWindowDoFn(
+                      FixedWindows.of(durationParameter(call.getOperands().get(2))),
+                      wmCol.getIndex(),
+                      outputSchema)))
+          .setRowSchema(outputSchema);
+    }
+  }
+
+  private Duration durationParameter(RexNode node) {
+    return Duration.millis(longValue(node));
+  }
+
+  private long longValue(RexNode operand) {
+    if (operand instanceof RexLiteral) {
+      return ((Number) RexLiteral.value(operand)).longValue();
+    } else {
+      throw new IllegalArgumentException(String.format("[%s] is not valid.", operand));
+    }
+  }
+
+  private static class FixedWindowDoFn extends DoFn<Row, Row> {
+    private int windowFieldIndex;

Review comment:
       all three of these should be `final`.

##########
File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/TVFStreamingUtils.java
##########
@@ -0,0 +1,24 @@
+/*
+ * 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.utils;
+
+/** Provides static constants or utils for TVF streaming. */
+public class TVFStreamingUtils {

Review comment:
       Nit: This seems like stuff that should go in `SqlWindowTableFunction`. Do these constants really need their own class?




----------------------------------------------------------------
This is an automated message from the 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