I've adjusted the permissions to allow for commenting.

On Thu, Feb 5, 2026 at 3:27 PM Danny McCormick via dev <[email protected]>
wrote:

> Would you mind opening up the doc for comments?
>
> At a high level, I'm skeptical of the pattern; it seems to me like it
> moves the burden of choosing the correct behavior from authors to consumers
> in non-obvious ways which range from harmless to potentially causing silent
> data loss. I think if a user wants to drop a PCollection, that should
> always be an active choice since the risk of data loss is much greater than
> the EoU benefit of extra code.
>
> To be concrete, we can look at the motivating examples:
>
> > A filter that routes rejects to a side output
>
> I'd argue that a better pattern than having a single transform which
> handles this is to either have a *Filter *or a *Partition* transform
> which a user can use as needed. These are different transforms because they
> have different purposes/core functionalities.
>
> If you really want to use a single transform to encapsulate the same core
> logic (while keeping the user convenience), you could do something like:
>
> ```
> good, bad = Filter(...).split_filtered_into_separate_collection()
> ```
>
> or
>
> ```
> pcoll | Partition(...).drop_non_main() | ChainedParDo() ...
> ```
>
> to allow chaining.
>
> > A parser that routes malformed input to a dead-letter output
> > A validator that routes violations separately
> > An enrichment that routes lookup failures aside
>
> These are the ones I'm really worried about. In all of these cases, we are
> silently dropping error output in a way that might be non-obvious to a
> user. As a user, if I use a parser that returns a single output, I would
> assume that any parse failures would lead to exceptions.
>
> With all that said, I am aligned with the goal of making pipelines like
> this easier to chain. Maybe an in between option would be adding a DoFn
> utility like:
>
> ```
> pcoll | Partition(...).keep_tag('main') | ChainedParDo()
> ```
>
> Where `keep_tag` forces an expansion where all tags other than main are
> dropped. What do you think?
>
> Thanks,
> Danny
>
> On Thu, Feb 5, 2026 at 3:04 PM Joey Tran <[email protected]>
> wrote:
>
>> Hey everyone,
>>
>> My team and I have been running into an awkward pattern with the python
>> and YAML SDK when we have transforms that have one "main" output that we
>> want to be able to ergonomically chain, and other "side" outputs that are
>> useful in some situations. I put together a brief design proposal for a new
>> PCollection type to make this easier - would appreciate any feedback or
>> thoughts. Open to different names as well.
>>
>> ForkedPCollection Design Doc
>> <https://docs.google.com/document/d/10kx8hVrF8JfdeIS6X1vjiADk0IRMTVup9aX3u-GthOo/edit?tab=t.0>
>>
>> Thanks!
>> Joey
>>
>> --
>>
>> Joey Tran | Staff Developer | AutoDesigner TL
>>
>> *he/him*
>>
>> [image: Schrödinger, Inc.] <https://schrodinger.com/>
>>
>

Reply via email to