One of my goals is to make the graph easier to read and map back to the SQL
EXPLAIN output. The way the graph is currently built (`toPTransform` vs
`toPCollection`) does make a big difference in that graph. I think it is
also important to have a common function to do the apply with consistent
naming. I think that will greatly help with ease of understanding. It
sounds like what really want is this in the BeamRelNode interface:

PInput buildPInput(Pipeline pipeline);
PTransform<PInput, PCollection<Row>> buildPTransform();

default PCollection<Row> toPCollection(Pipeline pipeline) {
    return buildPInput(pipeline).apply(getStageName(), buildPTransform());
}

Andrew

On Mon, Jun 11, 2018 at 2:27 PM Mingmin Xu <mingm...@gmail.com> wrote:

> EXPLAIN shows the execution plan in SQL perspective only. After converting
> to a Beam composite PTransform, there're more steps underneath, each Runner
> re-org Beam PTransforms again which makes the final pipeline hard to read.
> In SQL module itself, I don't see any difference between `toPTransform` and
> `toPCollection`. We could have an easy-to-understand step name when
> converting RelNodes, but Runners show the graph to developers.
>
> Mingmin
>
> On Mon, Jun 11, 2018 at 2:06 PM, Andrew Pilloud <apill...@google.com>
> wrote:
>
>> That sounds correct. And because each rel node might have a different
>> input there isn't a standard interface (like PTransform<PCollection<Row>,
>> PCollection<Row>> toPTransform());
>>
>> Andrew
>>
>> On Mon, Jun 11, 2018 at 1:31 PM Kenneth Knowles <k...@google.com> wrote:
>>
>>> Agree with that. It will be kind of tricky to generalize. I think there
>>> are some criteria in this case that might apply in other cases:
>>>
>>> 1. Each rel node (or construct of a DSL) should have a PTransform for
>>> how it computes its result from its inputs.
>>> 2. The inputs to that PTransform should actually be the inputs to the
>>> rel node!
>>>
>>> So I tried to improve #1 but I probably made #2 worse.
>>>
>>> Kenn
>>>
>>> On Mon, Jun 11, 2018 at 12:53 PM Anton Kedin <ke...@google.com> wrote:
>>>
>>>> Not answering the original question, but doesn't "explain" satisfy the
>>>> SQL use case?
>>>>
>>>> Going forward we probably want to solve this in a more general way. We
>>>> have at least 3 ways to represent the pipeline:
>>>>  - how runner executes it;
>>>>  - what it looks like when constructed;
>>>>  - what the user was describing in DSL;
>>>> And there will probably be more, if extra layers are built on top of
>>>> DSLs.
>>>>
>>>> If possible, we probably should be able to map any level of abstraction
>>>> to any other to better understand and debug the pipelines.
>>>>
>>>>
>>>> On Mon, Jun 11, 2018 at 12:17 PM Kenneth Knowles <k...@google.com>
>>>> wrote:
>>>>
>>>>> In other words, revert https://github.com/apache/beam/pull/4705/files,
>>>>> at least in spirit? I agree :-)
>>>>>
>>>>> Kenn
>>>>>
>>>>> On Mon, Jun 11, 2018 at 11:39 AM Andrew Pilloud <apill...@google.com>
>>>>> wrote:
>>>>>
>>>>>> We are currently converting the Calcite Rel tree to Beam by
>>>>>> recursively building a tree of nested PTransforms. This results in a 
>>>>>> weird
>>>>>> nested graph in the dataflow UI where each node contains its inputs 
>>>>>> nested
>>>>>> inside of it. I'm going to change the internal data structure for
>>>>>> converting the tree from a PTransform to a PCollection, which will result
>>>>>> in a more accurate representation of the tree structure being built and
>>>>>> should simplify the code as well. This will not change the public 
>>>>>> interface
>>>>>> to SQL, which will remain a PTransform. Any thoughts or objections?
>>>>>>
>>>>>> I was also wondering if there are tools for visualizing the Beam
>>>>>> graph aside from the dataflow runner UI. What other tools exist?
>>>>>>
>>>>>> Andrew
>>>>>>
>>>>>
>
>
> --
> ----
> Mingmin
>

Reply via email to