tysonjh commented on a change in pull request #10946: URL: https://github.com/apache/beam/pull/10946#discussion_r421656607
########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java ########## @@ -0,0 +1,157 @@ +/* + * 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; + +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: %s", Review comment: I don't know checkArgument too well, does it output the expected size? E.g. it would be nice if this error message said something like, %s expected 1 input but received %s ########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java ########## @@ -0,0 +1,157 @@ +/* + * 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; + +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: %s", + BeamTableFunctionScanRel.class.getSimpleName(), + input); + checkArgument( + ((RexCall) getCall()).getOperator().getName().equals("TUMBLE"), + "Only support TUMBLE table-valued function"); Review comment: Here too. It would be nice if the error message included the offending operator name. ########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java ########## @@ -0,0 +1,157 @@ +/* + * 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; + +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: %s", + BeamTableFunctionScanRel.class.getSimpleName(), + input); + checkArgument( + ((RexCall) getCall()).getOperator().getName().equals("TUMBLE"), + "Only support TUMBLE table-valued function"); + 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; + private FixedWindows windowFn; + private Schema outputSchema; + + public FixedWindowDoFn(FixedWindows windowFn, int windowFieldIndex, Schema schema) { + this.windowFn = windowFn; + this.windowFieldIndex = windowFieldIndex; + this.outputSchema = schema; + } + + @ProcessElement + public void processElement(ProcessContext c) { + Row row = c.element(); + IntervalWindow window = windowFn.assignWindow(row.getDateTime(windowFieldIndex).toInstant()); + Row.Builder builder = Row.withSchema(outputSchema); + builder.addValues(row.getValues()); + // TODO: add window_start and window_end; Review comment: If this isn't to be resolved in this PR, would having an associated Jira be appropriate? Or... is this TODO out of date? I see window start and end are used below. ########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java ########## @@ -0,0 +1,157 @@ +/* + * 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; + +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: %s", + BeamTableFunctionScanRel.class.getSimpleName(), + input); + checkArgument( + ((RexCall) getCall()).getOperator().getName().equals("TUMBLE"), + "Only support TUMBLE table-valued function"); + 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; + private FixedWindows windowFn; + private Schema outputSchema; + + public FixedWindowDoFn(FixedWindows windowFn, int windowFieldIndex, Schema schema) { + this.windowFn = windowFn; + this.windowFieldIndex = windowFieldIndex; + this.outputSchema = schema; + } + + @ProcessElement + public void processElement(ProcessContext c) { + Row row = c.element(); + IntervalWindow window = windowFn.assignWindow(row.getDateTime(windowFieldIndex).toInstant()); + Row.Builder builder = Row.withSchema(outputSchema); + builder.addValues(row.getValues()); + // TODO: add window_start and window_end; + builder.addValue(window.start()); + builder.addValue(window.end()); + c.output(builder.build()); + } + } + + @Override + public NodeStats estimateNodeStats(RelMetadataQuery mq) { + return BeamSqlRelUtils.getNodeStats(getInput(0), mq); + } + + @Override + public BeamCostModel beamComputeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) { + NodeStats inputEstimates = BeamSqlRelUtils.getNodeStats(getInput(0), mq); + + final double rowSize = getRowType().getFieldCount(); + final double cpu = inputEstimates.getRowCount() * inputEstimates.getRowCount() * rowSize; + final double cpuRate = inputEstimates.getRate() * inputEstimates.getWindow() * rowSize; Review comment: Can you explain the logic here? It may be worth adding a comment to describe the calculation. For example, how does cpu = row_count^2 * row_size ? It isn't obvious to me. ########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamTableFunctionScanRule.java ########## @@ -0,0 +1,60 @@ +/* + * 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.rule; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; +import org.apache.beam.sdk.extensions.sql.impl.rel.BeamTableFunctionScanRel; +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.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.TableFunctionScan; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalTableFunctionScan; + +public class BeamTableFunctionScanRule extends ConverterRule { Review comment: Please add a class comment. ########## File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlWindowTableFunction.java ########## @@ -0,0 +1,113 @@ +/* + * 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 java.util.ArrayList; +import java.util.List; +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.SqlCallBinding; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlFunction; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlFunctionCategory; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperandCountRange; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlOperandCountRanges; +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.type.SqlTypeName; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.validate.SqlValidator; + +public class SqlWindowTableFunction extends SqlFunction { Review comment: Please add class comment. ########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamTableFunctionScanRule.java ########## @@ -0,0 +1,60 @@ +/* + * 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.rule; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; +import org.apache.beam.sdk.extensions.sql.impl.rel.BeamTableFunctionScanRel; +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.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.TableFunctionScan; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalTableFunctionScan; + +public class BeamTableFunctionScanRule extends ConverterRule { + public static final BeamTableFunctionScanRule INSTANCE = new BeamTableFunctionScanRule(); + + private BeamTableFunctionScanRule() { + super( + LogicalTableFunctionScan.class, + Convention.NONE, + BeamLogicalConvention.INSTANCE, + "BeamTableFunctionScanRule"); + } + + @Override + public RelNode convert(RelNode relNode) { + TableFunctionScan tableFunctionScan = (TableFunctionScan) relNode; + // only support one input for table function scan. Review comment: Should this be a checkArgument? ########## File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java ########## @@ -551,6 +561,60 @@ public RexNode convertResolvedLiteral(ResolvedLiteral resolvedLiteral) { return ret; } + public RexCall convertTableValuedFunction( + RelNode input, + TableValuedFunction tvf, + List<ResolvedNodes.ResolvedTVFArgument> argumentList, + List<ResolvedColumn> inputTableColumns) { + switch (tvf.getName()) { + case "TUMBLE": + // TUMBLE tvf's second argument is descriptor. + ResolvedColumn wmCol = + extractWatermarkColumnFromDescriptor(argumentList.get(1).getDescriptorArg()); + if (wmCol.getType().getKind() != TYPE_TIMESTAMP) { + throw new IllegalArgumentException( + "Watermarked column should be TIMESTAMP type: " + + extractWatermarkColumnNameFromDescriptor( + argumentList.get(1).getDescriptorArg())); + } + return (RexCall) + rexBuilder() + .makeCall( + new SqlWindowTableFunction(SqlKind.TUMBLE.name()), + convertRelNodeToRexRangeRef(input), + convertWatermarkedResolvedColumnToRexInputRef(wmCol, inputTableColumns), + convertIntervalToRexIntervalLiteral( + (ResolvedLiteral) argumentList.get(2).getExpr())); + default: + throw new UnsupportedOperationException( + "Does not support table-valued function: " + tvf.getName()); + } + } + + private RexInputRef convertWatermarkedResolvedColumnToRexInputRef( + ResolvedColumn wmCol, List<ResolvedColumn> inputTableColumns) { + for (int i = 0; i < inputTableColumns.size(); i++) { + if (inputTableColumns.get(i).equals(wmCol)) { + return rexBuilder() + .makeInputRef(TypeUtils.toRelDataType(rexBuilder(), wmCol.getType(), false), i); + } + } + + // ZetaSQL parser guarantees that wmCol can be found from inputTableColumns. + // so it shouldn't reach here. + throw new IllegalArgumentException(); Review comment: Would this be more descriptive as an IllegalStateException? Either way having a message would be useful here. ########## File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java ########## @@ -551,6 +561,60 @@ public RexNode convertResolvedLiteral(ResolvedLiteral resolvedLiteral) { return ret; } + public RexCall convertTableValuedFunction( Review comment: Please add a method comment since it is public and not 'simple' like a getter or setter. ########## 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> { Review comment: Please add a class comment. They're really useful and should be on all classes and non-trivial public methods. ########## File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlWindowTableFunction.java ########## @@ -0,0 +1,113 @@ +/* + * 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 java.util.ArrayList; +import java.util.List; +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.SqlCallBinding; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlFunction; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlFunctionCategory; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperandCountRange; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlOperandCountRanges; +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.type.SqlTypeName; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.validate.SqlValidator; + +public class SqlWindowTableFunction extends SqlFunction { + public SqlWindowTableFunction(String name) { + super( + name, + SqlKind.OTHER_FUNCTION, + ARG0_TABLE_FUNCTION_WINDOWING, + null, + null, + SqlFunctionCategory.SYSTEM); + } + + @Override + public SqlOperandCountRange getOperandCountRange() { + return SqlOperandCountRanges.of(3); + } + + @Override + public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) { + // There should only be three operands, and number of operands are checked before + // this call. Review comment: For my own understanding, where are they checked? ########## 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: I've seen window_start and window_end come up a few times in this PR. Is there a reasonable place to have these as constants and would it make sense to do this? ########## File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java ########## @@ -551,6 +561,60 @@ public RexNode convertResolvedLiteral(ResolvedLiteral resolvedLiteral) { return ret; } + public RexCall convertTableValuedFunction( + RelNode input, + TableValuedFunction tvf, + List<ResolvedNodes.ResolvedTVFArgument> argumentList, + List<ResolvedColumn> inputTableColumns) { + switch (tvf.getName()) { + case "TUMBLE": + // TUMBLE tvf's second argument is descriptor. + ResolvedColumn wmCol = + extractWatermarkColumnFromDescriptor(argumentList.get(1).getDescriptorArg()); + if (wmCol.getType().getKind() != TYPE_TIMESTAMP) { + throw new IllegalArgumentException( + "Watermarked column should be TIMESTAMP type: " + + extractWatermarkColumnNameFromDescriptor( Review comment: Can you do wmCol.getName() or is this different? ---------------------------------------------------------------- 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: [email protected]
