[ 
https://issues.apache.org/jira/browse/BEAM-9363?focusedWorklogId=433961&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-433961
 ]

ASF GitHub Bot logged work on BEAM-9363:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 15/May/20 22:29
            Start Date: 15/May/20 22:29
    Worklog Time Spent: 10m 
      Work Description: kennknowles commented on a change in pull request 
#10946:
URL: https://github.com/apache/beam/pull/10946#discussion_r426073628



##########
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(

Review comment:
       My point is really that committing the code like this makes code with 
names that don't make sense. If you get interrupted to do something else after 
this PR, then the codebase will be in an incorrect state.




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


Issue Time Tracking
-------------------

    Worklog Id:     (was: 433961)
    Time Spent: 5h  (was: 4h 50m)

> BeamSQL windowing as TVF
> ------------------------
>
>                 Key: BEAM-9363
>                 URL: https://issues.apache.org/jira/browse/BEAM-9363
>             Project: Beam
>          Issue Type: New Feature
>          Components: dsl-sql, dsl-sql-zetasql
>            Reporter: Rui Wang
>            Assignee: Rui Wang
>            Priority: Major
>          Time Spent: 5h
>  Remaining Estimate: 0h
>
>  This Jira tracks the implementation for 
> https://s.apache.org/streaming-beam-sql
> TVF is table-valued function, which is a SQL feature that produce a table as 
> function's output.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to