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

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_r360590462
 
 

 ##########
 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() {
+    if (input instanceof BeamSortRel) {
+      return ((BeamSortRel) input).getCount();
+    }
+
+    throw new RuntimeException("Could not get the limit count from a non 
BeamSortRel input.");
+  }
+
+  @Override
+  public NodeStats estimateNodeStats(RelMetadataQuery mq) {
+    NodeStats inputStat = BeamSqlRelUtils.getNodeStats(this.input, mq);
+    double selectivity = estimateFilterSelectivity(getInput(), program, mq);
+
+    return inputStat.multiply(selectivity);
+  }
+
+  private static double estimateFilterSelectivity(
+      RelNode child, RexProgram program, RelMetadataQuery mq) {
+    // Similar to calcite, if the calc node is representing filter operation 
we estimate the filter
+    // selectivity based on the number of equality conditions, number of 
inequality conditions, ....
+    RexLocalRef programCondition = program.getCondition();
+    RexNode condition;
+    if (programCondition == null) {
+      condition = null;
+    } else {
+      condition = program.expandLocalRef(programCondition);
+    }
+    // Currently this gets the selectivity based on Calcite's Selectivity 
Handler (RelMdSelectivity)
+    return mq.getSelectivity(child, condition);
+  }
+
+  @Override
+  public BeamCostModel beamComputeSelfCost(RelOptPlanner planner, 
RelMetadataQuery mq) {
+    NodeStats inputStat = BeamSqlRelUtils.getNodeStats(this.input, mq);
+    return BeamCostModel.FACTORY.makeCost(inputStat.getRowCount(), 
inputStat.getRate());
+  }
+
+  @Override
+  public boolean isInputSortRelAndLimitOnly() {
+    return (input instanceof BeamSortRel) && ((BeamSortRel) 
input).isLimitOnly();
+  }
+
+  /**
+   * {@code CalcFn} is the executor for a {@link BeamZetaSqlCalcRel} step. The 
implementation is
+   * based on the {@code ZetaSQL} expression evaluator.
+   */
+  private static class CalcFn extends DoFn<Row, Row> {
+    private final List<String> projects;
+    @Nullable private final String condition;
+    private final Schema inputSchema;
+    private final Schema outputSchema;
+    private transient List<PreparedExpression> projectExps;
+    @Nullable private transient PreparedExpression conditionExp;
+
+    CalcFn(
+        List<String> projects,
+        @Nullable String condition,
+        Schema inputSchema,
+        Schema outputSchema) {
+      Preconditions.checkArgument(projects.size() == 
outputSchema.getFieldCount());
+      this.projects = ImmutableList.copyOf(projects);
+      this.condition = condition;
+      this.inputSchema = inputSchema;
+      this.outputSchema = outputSchema;
+    }
+
+    @Setup
+    public void setup() {
+      AnalyzerOptions options = SqlAnalyzer.initAnalyzerOptions();
+      for (Field field : inputSchema.getFields()) {
+        options.addExpressionColumn(
+            sanitize(field.getName()), 
ZetaSqlUtils.beamFieldTypeToZetaSqlType(field.getType()));
+      }
+
+      projectExps = new ArrayList<>();
 
 Review comment:
   `//TODO: Only use a single PreparedExpression for all condition and projects`
 
----------------------------------------------------------------
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: 361913)
    Time Spent: 1h 40m  (was: 1.5h)

> 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: 1h 40m
>  Remaining Estimate: 0h
>




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

Reply via email to