On Thu, Feb 12, 2026 at 8:23 PM Robert Bradshaw via dev <[email protected]>
wrote:

> On Thu, Feb 12, 2026 at 4:47 PM Valentyn Tymofieiev <[email protected]>
> wrote:
> >
> > >  Were I to do it again, I would have such transforms return a dict or
> named tuple (if all outputs are
> > meaningful) or an "augmented" PCollection (as has been proposed here)
> > when they are auxiliary (and preferably leave the decision up to the
> > DoFn implementor, not the caller).
> >
> > Regarding the "augmented PCollection" concept, would it be feasible to
> think of a design where every PCollection is implicitly a container that
> has side outputs? In this world, a standard PCollection is a the corner
> case with 0 side outputs. I wonder if this could help avoid introducing a
> new distinct type like PCollectionWithSideOutputs.
> >
>

Big +1 from me. I've been tripped up many times from `.with_outputs`
changing the result of a ParDo transform from a PCollection to a tuple, and
I've seen other users similarly confused.


> > Looking at the code snippet below
> >
> > results = (p | Create(...)
> >              | ParDo(...).with_outputs('side_output_tag',
> main='main_tag'))
> >
> > # This currently fails with _InvalidUnpickledPCollection errors
> > results | LogElements()
> >
> >
> > This code is failing, since I don't specify the main output, so I think
> Beam treats the DoOutputsTuple as an iterable of data elements (the
> PCollections themselves) and maybe tries to Create() a new PCollection from
> them. However I explicitly specify which output is main. What if
> DoOutputsTuple in this case supported chaining off the 'main' PColl in this
> case?
>
> Are there any PTransforms that accept a DoOutputsTuple? (Or, if there
> are, can we identify them?) This is the primary downside I see to this
> route.
>

I'm guessing there are probably PTransforms out there somewhere that rely
on this behavior at this point. But maybe we can sidestep backwards
compatibility and just add a new method to use "side outputs", e.g.
`.with_side_outputs`? I think the semantic difference between
`.with_outputs` and `.with_side_outputs` is relatively clear.


>
> > On Thu, Feb 12, 2026 at 2:52 PM Danny McCormick via dev <
> [email protected]> wrote:
> >>
> >> My preference would be enabling `pcoll | Partition(...)['main'] |
> ChainedParDo()`, but I think I'm currently the only one with significant
> objections - I tried to make time for someone to join my dissent :)
> >>
> >> Given that, I'm ok with proceeding with roughly the original proposal
> (factoring conversation in the doc); my only request would be that we
> document the transform in a way that clearly discourages putting
> error/exception outputs in the secondary PCollection, and makes it clear
> that this is primarily for use cases where the main PCollection is
> sufficient for most use cases.
>
>
When you say `document the transform`, what transform are you referring to?
Or do you mean putting a warning in the docstring of
PCollectionWithSideOutputs?


> +1
>
> >> On Tue, Feb 10, 2026 at 4:42 PM Joey Tran <[email protected]>
> wrote:
> >>>
> >>> Just want to bump this. In what direction should we go here?
> >>>
> >>> On Fri, Feb 6, 2026 at 5:49 PM Joey Tran <[email protected]>
> wrote:
> >>>>
> >>>>
> >>>>
> >>>> On Fri, Feb 6, 2026 at 5:43 PM Robert Bradshaw <[email protected]>
> wrote:
> >>>>>
> >>>>> On Fri, Feb 6, 2026 at 2:36 PM Joey Tran <[email protected]>
> wrote:
> >>>>> >
> >>>>> > On Fri, Feb 6, 2026 at 4:43 PM Danny McCormick <
> [email protected]> wrote:
> >>>>> >>
> >>>>> >> On Fri, Feb 6, 2026 at 4:22 PM Joey Tran <
> [email protected]> wrote:
> >>>>> >>>
> >>>>> >>> FWIW, much of the value of this proposal to me is the better
> readability from not having to consider multiple versions of transforms and
> not having to break up chains to extract main outputs. I appreciate though
> that we'd be making a trade-off of readability of the "sad path" for
> readability of the "happy path"
> >>>>> >>
> >>>>> >>
> >>>>> >> Yeah, that makes sense; what do you think of the other
> alternative mentioned as an option for optimizing for both kinds of
> readability? Specifically, allowing:
> >>>>> >>
> >>>>> >>    pcoll | Partition(...)['main'] | ChainedParDo()
> >>>>> >>
> >>>>> >> I guess the downside there is education (all pipeline authors
> need to know this is an option as opposed to only one expert transform
> author), but I'm curious if it is sufficient for your context.
> >>>>> >
> >>>>> > Is the suggestion here to implement `__getitem__` on
> PTransform/ParDo so a particular pcollection can be specified? This would
> definitely be an improvement from the current state. I think one further
> improvement would be if we could specify the pcollection by attribute
> rather than by key/string, so `Partition(...).main` instead, but that risks
> pcollection name and ptransform method collisions.
> >>>>> >
> >>>>> > I'm still partial toward the other suggestions, particularly
> towards implementing `PTransform.with_outputs`, but this is probably
> sufficient for my context.
> >>>>>
> >>>>> I'll admit that I'm actually not a fan of with_outputs(...). It's not
> >>>>> very dry--I'd rather the consumer decide what it wants to consume by
> >>>>> consuming it than have to also (redundantly) specify it on the
> >>>>> producer. I think it dates back to trying to copy java where the
> >>>>> return type needs to be a typed PValue. Were I to do it again, I
> would
> >>>>> have such transforms return a dict or named tuple (if all outputs are
> >>>>> meaningful) or an "augmented" PCollection (as has been proposed here)
> >>>>> when they are auxiliary (and preferably leave the decision up to the
> >>>>> DoFn implementor, not the caller).
> >>>>>
> >>>>> - Robert
> >>>>
> >>>>
> >>>> Ha, yeah I also don't find it the most intuitively named /
> parametrized. I usually need to look at it's documentation each time I need
> to use it.  Standardization is nice though.
>

Reply via email to