You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/05/03 22:45:00 UTC

[jira] [Work logged] (BEAM-3773) [SQL] Investigate JDBC interface for Beam SQL

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

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

                Author: ASF GitHub Bot
            Created on: 03/May/18 22:44
            Start Date: 03/May/18 22:44
    Worklog Time Spent: 10m 
      Work Description: kennknowles closed pull request #5173: [BEAM-3773][SQL] Add EnumerableConverter for JDBC support
URL: https://github.com/apache/beam/pull/5173
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java
index af74dae7510..8423ddaf153 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java
@@ -20,6 +20,7 @@
 import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
 import org.apache.beam.sdk.extensions.sql.impl.rule.BeamAggregationRule;
+import org.apache.beam.sdk.extensions.sql.impl.rule.BeamEnumerableConverterRule;
 import org.apache.beam.sdk.extensions.sql.impl.rule.BeamFilterRule;
 import org.apache.beam.sdk.extensions.sql.impl.rule.BeamIntersectRule;
 import org.apache.beam.sdk.extensions.sql.impl.rule.BeamJoinRule;
@@ -49,7 +50,8 @@
           BeamIntersectRule.INSTANCE,
           BeamMinusRule.INSTANCE,
           BeamUnionRule.INSTANCE,
-          BeamJoinRule.INSTANCE)
+          BeamJoinRule.INSTANCE,
+          BeamEnumerableConverterRule.INSTANCE)
     };
   }
 }
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java
new file mode 100644
index 00000000000..0ccc6c893a9
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java
@@ -0,0 +1,168 @@
+/*
+ * 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 com.google.common.base.Preconditions.checkArgument;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.MetricNameFilter;
+import org.apache.beam.sdk.metrics.MetricQueryResults;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.metrics.MetricsFilter;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.calcite.adapter.enumerable.EnumerableRel;
+import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
+import org.apache.calcite.adapter.enumerable.PhysType;
+import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterImpl;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+
+/**
+ * BeamRelNode to replace a {@code Enumerable} node.
+ */
+public class BeamEnumerableConverter extends ConverterImpl implements EnumerableRel {
+
+  private final PipelineOptions options = PipelineOptionsFactory.create();
+
+  public BeamEnumerableConverter(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
+    super(cluster, ConventionTraitDef.INSTANCE, traits, input);
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new BeamEnumerableConverter(getCluster(), traitSet, sole(inputs));
+  }
+
+  @Override
+  public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+    // This should always be a last resort.
+    return planner.getCostFactory().makeHugeCost();
+  }
+
+  @Override
+  public Result implement(EnumerableRelImplementor implementor, Prefer prefer) {
+    final BlockBuilder list = new BlockBuilder();
+    final RelDataType rowType = getRowType();
+    final PhysType physType =
+        PhysTypeImpl.of(implementor.getTypeFactory(), rowType, prefer.preferArray());
+    final Expression options = implementor.stash(this.options, PipelineOptions.class);
+    final Expression node = implementor.stash((BeamRelNode) getInput(), BeamRelNode.class);
+    list.add(Expressions.call(BeamEnumerableConverter.class, "toEnumerable", options, node));
+    return implementor.result(physType, list.toBlock());
+  }
+
+  public static Enumerable<Object> toEnumerable(PipelineOptions options, BeamRelNode node) {
+    if (node instanceof BeamIOSinkRel) {
+      return count(options, node);
+    }
+    return collect(options, node);
+  }
+
+  private static PipelineResult run(PipelineOptions options, BeamRelNode node,
+      DoFn<Row, Void> doFn) {
+    Pipeline pipeline = Pipeline.create(options);
+    PCollectionTuple.empty(pipeline)
+        .apply(node.toPTransform())
+        .apply(ParDo.of(doFn));
+    PipelineResult result = pipeline.run();
+    result.waitUntilFinish();
+    return result;
+  }
+
+  private static Enumerable<Object> collect(PipelineOptions options, BeamRelNode node) {
+    long id = options.getOptionsId();
+    Queue<Object> values = new ConcurrentLinkedQueue<Object>();
+
+    checkArgument(options.getRunner().getCanonicalName()
+        .equals("org.apache.beam.runners.direct.DirectRunner"));
+    Collector.globalValues.put(id, values);
+    run(options, node, new Collector());
+    Collector.globalValues.remove(id);
+
+    return Linq4j.asEnumerable(values);
+  }
+
+  private static class Collector extends DoFn<Row, Void> {
+    // This will only work on the direct runner.
+    private static final Map<Long, Queue<Object>> globalValues =
+        new ConcurrentHashMap<Long, Queue<Object>>();
+
+    @Nullable
+    private volatile Queue<Object> values;
+
+    @StartBundle
+    public void startBundle(StartBundleContext context) {
+      long id = context.getPipelineOptions().getOptionsId();
+      values = globalValues.get(id);
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext context) {
+      Object[] input = context.element().getValues().toArray();
+      if (input.length == 1) {
+        values.add(input[0]);
+      } else {
+        values.add(input);
+      }
+    }
+  }
+
+  private static Enumerable<Object> count(PipelineOptions options, BeamRelNode node) {
+    PipelineResult result = run(options, node, new RowCounter());
+    MetricQueryResults metrics = result.metrics().queryMetrics(MetricsFilter.builder()
+        .addNameFilter(MetricNameFilter.named(BeamEnumerableConverter.class, "rows"))
+        .build());
+    long count = metrics.getCounters().iterator().next().getAttempted();
+    return Linq4j.singletonEnumerable(count);
+  }
+
+  private static class RowCounter extends DoFn<Row, Void> {
+    final Counter rows =
+        Metrics.counter(BeamEnumerableConverter.class, "rows");
+
+    @ProcessElement
+    public void processElement(ProcessContext context) {
+      rows.inc();
+    }
+  }
+}
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamEnumerableConverterRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamEnumerableConverterRule.java
new file mode 100644
index 00000000000..aceaedd5271
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamEnumerableConverterRule.java
@@ -0,0 +1,44 @@
+/*
+ * 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 org.apache.beam.sdk.extensions.sql.impl.rel.BeamEnumerableConverter;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+
+/**
+ * A {@code ConverterRule} to Convert {@link BeamRel} to {@link EnumerableConvention}.
+ *
+ */
+public class BeamEnumerableConverterRule extends ConverterRule {
+  public static final BeamEnumerableConverterRule INSTANCE = new BeamEnumerableConverterRule();
+
+  private BeamEnumerableConverterRule() {
+    super(RelNode.class, BeamLogicalConvention.INSTANCE,
+        EnumerableConvention.INSTANCE, "BeamEnumerableConverterRule");
+  }
+
+  @Override
+  public RelNode convert(RelNode rel) {
+    RelTraitSet newTraitSet = rel.getTraitSet().replace(getOutTrait());
+    return new BeamEnumerableConverter(rel.getCluster(), newTraitSet, rel);
+  }
+}
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverterTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverterTest.java
new file mode 100644
index 00000000000..c5f1b4be9c2
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverterTest.java
@@ -0,0 +1,159 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import com.google.common.collect.ImmutableList;
+import java.math.BigDecimal;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.sql.BeamSqlTable;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.impl.schema.BeamIOType;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+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.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.volcano.VolcanoPlanner;
+import org.apache.calcite.prepare.RelOptTableImpl;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexLiteral;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamEnumerableConverter}.
+ */
+public class BeamEnumerableConverterTest {
+  static RexBuilder rexBuilder = new RexBuilder(BeamQueryPlanner.TYPE_FACTORY);
+  static PipelineOptions options = PipelineOptionsFactory.create();
+
+  @Test
+  public void testToEnumerable_collectSingle() {
+    Schema schema = Schema.builder().addInt64Field("id", false).build();
+    RelDataType type = CalciteUtils.toCalciteRowType(schema, BeamQueryPlanner.TYPE_FACTORY);
+    ImmutableList<ImmutableList<RexLiteral>> tuples =
+        ImmutableList.of(ImmutableList.of(rexBuilder.makeBigintLiteral(BigDecimal.ZERO)));
+    BeamRelNode node = new BeamValuesRel(null, type, tuples, null);
+
+    Enumerable<Object> enumerable = BeamEnumerableConverter.toEnumerable(options, node);
+    Enumerator<Object> enumerator = enumerable.enumerator();
+
+    assertTrue(enumerator.moveNext());
+    assertEquals(0L, enumerator.current());
+    assertFalse(enumerator.moveNext());
+    enumerator.close();
+  }
+
+  @Test
+  public void testToEnumerable_collectMultiple() {
+    Schema schema =
+        Schema.builder().addInt64Field("id", false).addInt64Field("otherid", false).build();
+    RelDataType type = CalciteUtils.toCalciteRowType(schema, BeamQueryPlanner.TYPE_FACTORY);
+    ImmutableList<ImmutableList<RexLiteral>> tuples =
+        ImmutableList.of(
+            ImmutableList.of(
+                rexBuilder.makeBigintLiteral(BigDecimal.ZERO),
+                rexBuilder.makeBigintLiteral(BigDecimal.ONE)));
+    BeamRelNode node = new BeamValuesRel(null, type, tuples, null);
+
+    Enumerable<Object> enumerable = BeamEnumerableConverter.toEnumerable(options, node);
+    Enumerator<Object> enumerator = enumerable.enumerator();
+
+    assertTrue(enumerator.moveNext());
+    Object[] row = (Object[]) enumerator.current();
+    assertEquals(2, row.length);
+    assertEquals(0L, row[0]);
+    assertEquals(1L, row[1]);
+    assertFalse(enumerator.moveNext());
+    enumerator.close();
+  }
+
+  private static class FakeTable implements BeamSqlTable {
+    public BeamIOType getSourceType() {
+      return null;
+    }
+
+    public PCollection<Row> buildIOReader(Pipeline pipeline) {
+      return null;
+    }
+
+    public PTransform<? super PCollection<Row>, POutput> buildIOWriter() {
+      return new FakeIOWriter();
+    }
+
+    public Schema getSchema() {
+      return null;
+    }
+  }
+
+  private static class FakeIOWriter extends PTransform<PCollection<Row>, POutput> {
+    @Override
+    public POutput expand(PCollection<Row> input) {
+      input.apply(
+          ParDo.of(
+              new DoFn<Row, Void>() {
+                @ProcessElement
+                public void processElement(ProcessContext context) {}
+              }));
+      return PDone.in(input.getPipeline());
+    }
+  }
+
+  @Test
+  public void testToEnumerable_count() {
+    Schema schema = Schema.builder().addInt64Field("id", false).build();
+    RelDataType type = CalciteUtils.toCalciteRowType(schema, BeamQueryPlanner.TYPE_FACTORY);
+    ImmutableList<ImmutableList<RexLiteral>> tuples =
+        ImmutableList.of(
+            ImmutableList.of(rexBuilder.makeBigintLiteral(BigDecimal.ZERO)),
+            ImmutableList.of(rexBuilder.makeBigintLiteral(BigDecimal.ONE)));
+    BeamRelNode node =
+        new BeamIOSinkRel(
+            RelOptCluster.create(new VolcanoPlanner(), rexBuilder),
+            RelOptTableImpl.create(null, type, ImmutableList.of(), null),
+            null,
+            new BeamValuesRel(null, type, tuples, null),
+            null,
+            null,
+            null,
+            false,
+            new FakeTable());
+
+    Enumerable<Object> enumerable = BeamEnumerableConverter.toEnumerable(options, node);
+    Enumerator<Object> enumerator = enumerable.enumerator();
+
+    assertTrue(enumerator.moveNext());
+    assertEquals(2L, enumerator.current());
+    assertFalse(enumerator.moveNext());
+    enumerator.close();
+  }
+}


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


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

    Worklog Id:     (was: 98066)
    Time Spent: 6h 10m  (was: 6h)

> [SQL] Investigate JDBC interface for Beam SQL
> ---------------------------------------------
>
>                 Key: BEAM-3773
>                 URL: https://issues.apache.org/jira/browse/BEAM-3773
>             Project: Beam
>          Issue Type: Improvement
>          Components: dsl-sql
>            Reporter: Anton Kedin
>            Assignee: Andrew Pilloud
>            Priority: Major
>          Time Spent: 6h 10m
>  Remaining Estimate: 0h
>
> JDBC allows integration with a lot of third-party tools, e.g [Zeppelin|https://zeppelin.apache.org/docs/0.7.0/manual/interpreters.html], [sqlline|https://github.com/julianhyde/sqlline]. We should look into how feasible it is to implement a JDBC interface for Beam SQL



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)