Interesting point, though, that a large amount of existing display data is
used by zero runners. So the value proposition is still mostly an untested
hypothesis.

Kenn

On Thu, May 13, 2021 at 4:50 PM Robert Bradshaw <[email protected]> wrote:

> +1, definitely think display data more often belongs to composites
> than leafs. Dataflow is moving to a model where it accepts Beam protos
> directly; hopefully we can get that information to the UI.
>
> On Thu, May 13, 2021 at 4:47 PM Valentyn Tymofieiev <[email protected]>
> wrote:
> >
> > I also happened to look at display data associated with Beam BigQuery
> IOs. In my opinion,  for IO 'display data' bits to be useful, they need to
> be visualized at the top-level (composite) transforms.  BQ IOs are one of
> the most complex transforms in Beam and generate very involved graphs.
> Display data information becomes too hard to find within the graph.
> >
> > On Wed, May 12, 2021 at 9:21 AM Reuven Lax <[email protected]> wrote:
> >>
> >> This is arguably a bug in Dataflow's backend. The backend only knows
> about primitive operations (ParDo, Flatten, etc.), and doesn't currently
> model a PTransform as an independent entity. Rather it infers the existence
> of the PTransform based on the naming of the operations (i.e. if you have
> operations named a/b and a/c, you infer a PTransform named a containing b
> and c). This is how the Dataflow UI knows how to display composite
> transforms.
> >>
> >> Should Google support PTransforms as a top-level object? Yes - as you
> noticed this is an easy way to trip up, and sometimes innocent-seeming
> refactoring can cause display data to get "lost." I'm not sure what the
> current priority of this bug is, and it may not be fixed until things are
> fully on portable pipelines. For now, I suggest putting display data on
> primitive operations.
> >>
> >> Reuven
> >>
> >> On Wed, May 12, 2021 at 7:10 AM Ismaël Mejía <[email protected]> wrote:
> >>>
> >>> Running a pipeline on Dataflow I noticed it was not showing the
> 'display data' of ParquetIO on the Dataflow UI, after digging deeper I
> found that composite transforms are not shown on Dataflow.
> >>>
> >>> BEAM-366 Support Display Data on Composite Transforms
> >>> https://issues.apache.org/jira/browse/BEAM-366
> >>>
> >>> I also noticed that for primitive transforms what is shown is not the
> populateDisplayData code extended from PTransform but the
> populateDisplayData method code implemented at the parametrizing function
> level, concretely the DoFn or Source for the case of IOs.
> >>>
> >>> This of course surprised me because we have been implemented all these
> methods in the wrong place (at the PTransform level) for years and ignoring
> the function so they are not shown in the UI, so I was wondering:
> >>>
> >>> 1. Does Google plan to support displaying composite transforms
> (BEAM-366) at some point?
> >>>
> >>> 2. If (1) is not happening soon, shall we refine all our
> populateDisplayData implementations to be done only at the Function level
> (DoFn, Source, WindowFn)?
> >>>
> >>> Since Open Source runners (Flink, Spark, etc) do not use DisplayData
> at all I suppose we should keep this discussion at the Dataflow level only
> at this time.
> >>>
> >>> I ignore how this is modeled on Portable Pipelines, is DisplayData
> part of FunctionSpec to support the current use case? I saw that
> DisplayData is considered at the PTransform level so this should cover the
> Composite case, so I am curious if we are considering the parametrized
> function level currently in use correctly for Portable pipelines.
> >>>
>

Reply via email to