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

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

                Author: ASF GitHub Bot
            Created on: 19/Jun/18 20:45
            Start Date: 19/Jun/18 20:45
    Worklog Time Spent: 10m 
      Work Description: kennknowles commented on a change in pull request 
#5673: [BEAM-4575] Cleanly transform graph from Calcite to Beam SQL
URL: https://github.com/apache/beam/pull/5673#discussion_r196571844
 
 

 ##########
 File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java
 ##########
 @@ -17,17 +17,34 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.rel;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.Row;
 import org.apache.calcite.rel.RelNode;
 
 /** A {@link RelNode} that can also give a {@link PTransform} that implements 
the expression. */
 public interface BeamRelNode extends RelNode {
-  /**
-   * A {@link BeamRelNode} is a recursive structure, the {@code 
BeamQueryPlanner} visits it with a
-   * DFS(Depth-First-Search) algorithm.
-   */
-  PTransform<PCollectionTuple, PCollection<Row>> toPTransform();
+  /** Transforms the inputs into a PInput. */
+  default PInput buildPInput(Pipeline pipeline, Map<Integer, PCollection<Row>> 
cache) {
+    List<RelNode> inputs = getInputs();
+    if (inputs.size() == 0) {
+      return pipeline.begin();
+    }
+    List<PCollection<Row>> pInputs = new ArrayList(inputs.size());
+    for (RelNode input : inputs) {
+      pInputs.add(BeamSqlRelUtils.toPCollection(pipeline, (BeamRelNode) input, 
cache));
+    }
+    if (pInputs.size() == 1) {
+      return pInputs.get(0);
+    }
+    return PCollectionList.of(pInputs);
+  }
+
+  PTransform<PInput, PCollection<Row>> buildPTransform();
 
 Review comment:
   I think you made a very good case that the prior API is not quite right. 
Something bugs me about this proposed API too. Here's where I'm coming from now:
   
   1. `BeamRelNode` instance corresponds to a `PCollection` instance (committed 
to particular inputs)
   2. `BeamRelNode` type (plus maybe some side conditions) corresponds to a 
not-yet-applied `PTransform` (not yet committed to particular inputs)
   
   So I was expecting this PR to be a return to 1 with a single method 
`BeamRelNode.toPCollection`, or some such. The difference in failure modes is:
   
    - `toPTransform`: PTransform that ignores its inputs
    - `toPCollection`: no PTransform encapsulating the rel's logic (could also 
have a PTransform that ignores its inputs)
   
   To ensure the first, you want whoever is responsible for mapping to a 
`PTransform` to have no access to the `Rel` instance. To ensure the second, you 
want to make sure the thing building the `PCollection` is obligated to just 
pass its (recursively converted) inputs to some `PTransform` without any other 
ad hoc logic.
   
   Here's an idea:
   
   1. `BeamRelNode.toPCollection` that does ad hoc logic on its recursively 
computed inputs, not worrying about a `PTransform`.
   2. `BeamSqlRelUtils` that does the recursive traversal and calls each 
`toPCollection` from within a `PTransform.expand` that it makes right on the 
spot.

----------------------------------------------------------------
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:
[email protected]


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

    Worklog Id:     (was: 113366)
    Time Spent: 2h 50m  (was: 2h 40m)

> Beam SQL should cleanly transform graph from Calcite
> ----------------------------------------------------
>
>                 Key: BEAM-4575
>                 URL: https://issues.apache.org/jira/browse/BEAM-4575
>             Project: Beam
>          Issue Type: New Feature
>          Components: dsl-sql
>            Reporter: Andrew Pilloud
>            Assignee: Andrew Pilloud
>            Priority: Major
>          Time Spent: 2h 50m
>  Remaining Estimate: 0h
>
> It would be nice if the Beam graph matched the Calcite graph in structure 
> with each node generating a PTransform that is applied onto the PCollection 
> of it's parent. We should also ensure that each Calcite node only appears in 
> the Beam graph one time.



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

Reply via email to