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

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

                Author: ASF GitHub Bot
            Created on: 20/Dec/19 22:20
            Start Date: 20/Dec/19 22:20
    Worklog Time Spent: 10m 
      Work Description: apilloud commented on pull request #9913: [BEAM-8630] 
Prototype of BeamZetaSqlCalcRel
URL: https://github.com/apache/beam/pull/9913#discussion_r360588881
 
 

 ##########
 File path: 
sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ * 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 com.google.zetasql.AnalyzerOptions;
+import com.google.zetasql.PreparedExpression;
+import com.google.zetasql.Value;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.IntFunction;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Internal;
+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.rel.BeamRelNode;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSortRel;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils;
+import org.apache.beam.sdk.extensions.sql.impl.rel.WithLimitableInput;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
+import 
org.apache.beam.sdk.extensions.sql.meta.provider.bigquery.BeamBigQuerySqlDialect;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.Field;
+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.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.Calc;
+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.rel2sql.SqlImplementor;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLocalRef;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexProgram;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlDialect;
+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.SqlNode;
+import 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+
+/**
+ * BeamRelNode to replace {@code Project} and {@code Filter} node based on the 
{@code ZetaSQL}
+ * expression evaluator.
+ */
+// TODO[BEAM-8630]: This class is currently a prototype and not used in 
runtime.
+@Internal
+public class BeamZetaSqlCalcRel extends Calc implements BeamRelNode, 
WithLimitableInput {
+
+  private static final SqlDialect DIALECT = BeamBigQuerySqlDialect.DEFAULT;
+  private final SqlImplementor.Context context;
+
+  public BeamZetaSqlCalcRel(
+      RelOptCluster cluster, RelTraitSet traits, RelNode input, RexProgram 
program) {
+    super(cluster, traits, input, program);
+    final IntFunction<SqlNode> fn =
+        i ->
+            new SqlIdentifier(
+                
getProgram().getInputRowType().getFieldList().get(i).getName(), 
SqlParserPos.ZERO);
+    context = new SqlImplementor.SimpleContext(DIALECT, fn);
+  }
+
+  @Override
+  public Calc copy(RelTraitSet traitSet, RelNode input, RexProgram program) {
+    return new BeamZetaSqlCalcRel(getCluster(), traitSet, input, program);
+  }
+
+  @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> pinput) {
+      Preconditions.checkArgument(
+          pinput.size() == 1,
+          "%s expected a single input PCollection, but received %d.",
+          BeamZetaSqlCalcRel.class.getSimpleName(),
+          pinput.size());
+      PCollection<Row> upstream = pinput.get(0);
+
+      final List<String> projects =
+          getProgram().getProjectList().stream()
+              .map(BeamZetaSqlCalcRel.this::unparseRexNode)
+              .collect(Collectors.toList());
+      final RexNode condition = getProgram().getCondition();
+
+      // TODO[BEAM-8630]: validate sql expressions at pipeline construction 
time
+      Schema outputSchema = CalciteUtils.toSchema(getRowType());
+      CalcFn calcFn =
+          new CalcFn(
+              projects,
+              condition == null ? null : unparseRexNode(condition),
+              upstream.getSchema(),
+              outputSchema);
+      return upstream.apply(ParDo.of(calcFn)).setRowSchema(outputSchema);
+    }
+  }
+
+  private String unparseRexNode(RexNode rex) {
+    return context.toSql(getProgram(), rex).toSqlString(DIALECT).getSql();
+  }
+
+  @Override
+  public int getLimitCountOfSortRel() {
 
 Review comment:
   Looks like everything from here to `isInputSortRelAndLimitOnly` is 
boilerplate that should be common between the two implementations.
 
----------------------------------------------------------------
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]


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

    Worklog Id:     (was: 361911)
    Time Spent: 1.5h  (was: 1h 20m)

> Prototype of BeamSQL Calc using ZetaSQL Expression Evaluator
> ------------------------------------------------------------
>
>                 Key: BEAM-8630
>                 URL: https://issues.apache.org/jira/browse/BEAM-8630
>             Project: Beam
>          Issue Type: New Feature
>          Components: dsl-sql
>            Reporter: Yueyang Qiu
>            Assignee: Yueyang Qiu
>            Priority: Major
>          Time Spent: 1.5h
>  Remaining Estimate: 0h
>




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

Reply via email to