E.g. something like https://github.com/apache/beam/pull/11283
On Wed, Apr 1, 2020 at 2:57 PM Robert Bradshaw <[email protected]> wrote: > On Wed, Apr 1, 2020 at 1:48 PM Sam Rohde <[email protected]> wrote: > >> To restate the original issue it is that the current method of setting >> the output tags on PCollections from composites drops the tag information >> of the returned PCollections. >> > > Composite PTransforms should *not* be setting output tags on > returned PCollecitons; this will break producing outputs from the actual > primitive that produces them. > > >> So an expand returning a dict will have its outputs labeled as None, 1, >> ..., len(outputs). This is broken because embedded payloads in composites >> won't be able to reference the outputs if they differ from the returned >> value. >> > > Yes, we need a way for composites to declare their output tags. Currently > this is only supported for the multi-output ParDo primitive. > > >> In this case, having the restriction of no nesting decreases technical >> complexity substantially and always giving the tag unambiguously informs >> the SDK what to name the outputs. We can also allow for arbitrary nesting, >> we'll just have to figure out an unambiguous naming scheme for the >> PCollections. >> > > How about this: if the returned PValue is a dictionary of string -> > PCollection, we use the keys as tags. We can extend this naturally to > tuples, named tuples, nesting, etc. (though I don't know if there are any > hidden assumptions left about having an output labeled None if we want to > push this through to completion). > > >> >> >> >> On Wed, Apr 1, 2020 at 12:44 PM Robert Bradshaw <[email protected]> >> wrote: >> >>> I'm -1 on this, it adds additional restrictions and I don't see what >>> this buys us. (In particular, it doesn't address the original issue.) >>> >>> On Wed, Apr 1, 2020 at 12:41 PM Sam Rohde <[email protected]> wrote: >>> >>>> So then how does the proposal sound? >>>> >>>> Writing again here: >>>> PTransform.expand: (...) -> Union[PValue, NamedTuple[str, PCollection], >>>> Tuple[str, PCollection], Dict[str, PCollection], DoOutputsTuple] >>>> >>>> i.e. no arbitrary nesting when outputting from an expand >>>> >>>> On Tue, Mar 31, 2020 at 5:15 PM Robert Bradshaw <[email protected]> >>>> wrote: >>>> >>>>> On Tue, Mar 31, 2020 at 4:13 PM Luke Cwik <[email protected]> wrote: >>>>> > >>>>> > It is important that composites know how things are named so that >>>>> any embedded payloads in the composite PTransform can reference the >>>>> outputs >>>>> appropriately. >>>>> >>>>> Very good point. This is part of the cleanup to treat inputs and >>>>> outputs of PCollections as maps rather than lists generally across the >>>>> Python representations (which also relates to some of the ugliness >>>>> that Cham has been running into with cross-language). >>>>> >>>>> > On Tue, Mar 31, 2020 at 2:51 PM Robert Bradshaw <[email protected]> >>>>> wrote: >>>>> >> >>>>> >> On Tue, Mar 31, 2020 at 1:13 PM Sam Rohde <[email protected]> >>>>> wrote: >>>>> >> >>> >>>>> >> >>> * Don't allow arbitrary nestings returned during expansion, >>>>> force composite transforms to always provide an unambiguous name (either a >>>>> tuple with PCollections with unique tags or a dictionary with untagged >>>>> PCollections or a singular PCollection (Java and Go SDKs do this)). >>>>> >> >> >>>>> >> >> I believe that aligning with Java and Go would be the right way >>>>> to go here. I don't know if this would limit expressiveness. >>>>> >> > >>>>> >> > Yeah this sounds like a much more elegant way of handling this >>>>> situation. I would lean towards this limiting expressiveness because there >>>>> would be a limit to nesting, but I think that the trade-off with reducing >>>>> complexity is worth it. >>>>> >> > >>>>> >> > So in summary it could be: >>>>> >> > PTransform.expand: (...) -> Union[PValue, NamedTuple[str, >>>>> PCollection], Tuple[str, PCollection], Dict[str, PCollection], >>>>> DoOutputsTuple] >>>>> >> > >>>>> >> > With the expectation that (pseudo-code): >>>>> >> > a_transform = ATransform() >>>>> >> > >>>>> ATransform.from_runner_api(a_transform.to_runner_api()).outputs.keys() == >>>>> a_transform.outputs.keys() >>>>> >> > >>>>> >> > Since this changes the Python SDK composite transform API, what >>>>> would be the next steps for the community to come to a consensus on this? >>>>> >> >>>>> >> It seems here we're conflating the nesting of PValue results with >>>>> the >>>>> >> nesting of composite operations. >>>>> >> >>>>> >> Both examples in the original post have PTransform nesting (a >>>>> >> composite) returning a flat tuple. This is completely orthogonal to >>>>> >> the idea of a PTransform returning a nested result (such as (pc1, >>>>> >> (pc2, pc3))) and forbidding the latter won't solve the former. >>>>> >> >>>>> >> Currently, with the exception of explicit names given for >>>>> multi-output >>>>> >> ParDos, we simply label the outputs sequentially with 0, 1, 2, 3, >>>>> ... >>>>> >> (Actually, for historical reasons, it's None, 1, 2, 3, ...), no >>>>> matter >>>>> >> the nesting. We could do better, e.g. for the example above, label >>>>> >> them "0", "1.0", "1.1", or use the keys in the returned dict, but >>>>> this >>>>> >> is separate from the idea of trying to relate the output tags of >>>>> >> composites to the output tags of their inner transforms. >>>>> >> >>>>> >> - Robert >>>>> >>>>
