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

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

                Author: ASF GitHub Bot
            Created on: 20/Apr/18 03:41
            Start Date: 20/Apr/18 03:41
    Worklog Time Spent: 10m 
      Work Description: kennknowles commented on a change in pull request 
#5173: [BEAM-3773][SQL] Add EnumerableConverter for JDBC support
URL: https://github.com/apache/beam/pull/5173#discussion_r182939167
 
 

 ##########
 File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java
 ##########
 @@ -0,0 +1,160 @@
+/*
+ * 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 java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+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 {
+
+  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 node = implementor.stash((BeamRelNode) getInput(), 
BeamRelNode.class);
+    list.add(Expressions.call(BeamEnumerableConverter.class, "toEnumerable", 
node));
+    return implementor.result(physType, list.toBlock());
+  }
+
+  public static Enumerable<Object> toEnumerable(BeamRelNode node) {
+    PipelineOptions options = PipelineOptionsFactory.create();
+    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>();
+
+    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.
 
 Review comment:
   It seems like there are two routes that are interesting here: (1) expose the 
ability to observe PCollection contents in the DirectRunner. This used to be 
the case when it only supported bounded collections. It is a different 
situation now, as the contents are never materialized. But that could change.
   
   The other thing that would potentially make it cross runner, and what Scio 
does, is to write it to a sink. It could be TextIO writing to the 
globally-required tempLocation. We should definitely learn from Scio either way.

----------------------------------------------------------------
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:
us...@infra.apache.org


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

    Worklog Id:     (was: 93057)
    Time Spent: 1h  (was: 50m)

> [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: 1h
>  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)

Reply via email to