Huh. This used to be a hard error in Java, but I guess it's togglable with an option now. We should probably add the option to toggle Python too. (Unclear what the default should be, but this probably ties into re-thinking how pipeline update should work.)
On Thu, Oct 5, 2023 at 4:58 AM Joey Tran <joey.t...@schrodinger.com> wrote: > Makes sense that the requirement is the same, but is the label > auto-generation behavior the same? I modified the BeamJava > wordcount example[1] to do the regex filter twice in a row, and unlike the > BeamPython example I posted before, it just warns instead of throwing an > exception. > > Tangentially, is it expected that the Beam playground examples don't have > a way to see the outputs of a run example? I have a vague memory that there > used to be a way to navigate to an output file after it's generated but not > sure if I just dreamt that up. Playing with the examples, I wasn't positive > if my runs were actually succeeding or not based on the stdout alone. > > [1] https://play.beam.apache.org/?sdk=java&shared=mI7WUeje_r2 > <https://play.beam.apache.org/?sdk=java&shared=mI7WUeje_r2> > [2] https://play.beam.apache.org/?sdk=python&shared=hIrm7jvCamW > > On Wed, Oct 4, 2023 at 12:16 PM Robert Bradshaw via user < > u...@beam.apache.org> wrote: > >> BeamJava and BeamPython have the exact same behavior: transform names >> within must be distinct [1]. This is because we do not necessarily know at >> pipeline construction time if the pipeline will be streaming or batch, or >> if it will be updated in the future, so the decision was made to impose >> this restriction up front. Both will auto-generate a name for you if one is >> not given, but will do so deterministically (not depending on some global >> context) to avoid potential update problems. >> >> [1] Note that this applies to the fully qualified transform name, so the >> naming only has to be distinct within a composite transform (or at the top >> level--the pipeline itself is isomorphic to a single composite transform). >> >> On Wed, Oct 4, 2023 at 3:43 AM Joey Tran <joey.t...@schrodinger.com> >> wrote: >> >>> Cross posting this thread to dev@ to see if this is intentional >>> behavior or if it's something worth changing for the python SDK >>> >>> On Tue, Oct 3, 2023, 10:10 PM XQ Hu via user <u...@beam.apache.org> >>> wrote: >>> >>>> That suggests the default label is created as that, which indeed causes >>>> the duplication error. >>>> >>>> On Tue, Oct 3, 2023 at 9:15 PM Joey Tran <joey.t...@schrodinger.com> >>>> wrote: >>>> >>>>> Not sure what that suggests >>>>> >>>>> On Tue, Oct 3, 2023, 6:24 PM XQ Hu via user <u...@beam.apache.org> >>>>> wrote: >>>>> >>>>>> Looks like this is the current behaviour. If you have `t = >>>>>> beam.Filter(identity_filter)`, `t.label` is defined as >>>>>> `Filter(identity_filter)`. >>>>>> >>>>>> On Mon, Oct 2, 2023 at 9:25 AM Joey Tran <joey.t...@schrodinger.com> >>>>>> wrote: >>>>>> >>>>>>> You don't have to specify the names if the callable you pass in is >>>>>>> /different/ for two `beam.Map`s, but if the callable is the same you >>>>>>> must >>>>>>> specify a label. For example, the below will raise an exception: >>>>>>> >>>>>>> ``` >>>>>>> | beam.Filter(identity_filter) >>>>>>> | beam.Filter(identity_filter) >>>>>>> ``` >>>>>>> >>>>>>> Here's an example on playground that shows the error message you get >>>>>>> [1]. I marked every line I added with a "# ++". >>>>>>> >>>>>>> It's a contrived example, but using a map or filter at the same >>>>>>> pipeline level probably comes up often, at least in my inexperience. For >>>>>>> example, you. might have a pipeline that partitions a pcoll into three >>>>>>> different pcolls, runs some transforms on them, and then runs the same >>>>>>> type >>>>>>> of filter on each of them. >>>>>>> >>>>>>> The case that happens most often for me is using the `assert_that` >>>>>>> [2] testing transform. In this case, I think often users will really >>>>>>> have >>>>>>> no need for a disambiguating label as they're often just writing unit >>>>>>> tests >>>>>>> that test a few different properties of their workflow. >>>>>>> >>>>>>> [1] https://play.beam.apache.org/?sdk=python&shared=hIrm7jvCamW >>>>>>> [2] >>>>>>> https://beam.apache.org/releases/pydoc/2.29.0/apache_beam.testing.util.html#apache_beam.testing.util.assert_that >>>>>>> >>>>>>> On Mon, Oct 2, 2023 at 9:08 AM Bruno Volpato via user < >>>>>>> u...@beam.apache.org> wrote: >>>>>>> >>>>>>>> If I understand the question correctly, you don't have to specify >>>>>>>> those names. >>>>>>>> >>>>>>>> As Reuven pointed out, it is probably a good idea so you have a >>>>>>>> stable / deterministic graph. >>>>>>>> But in the Python SDK, you can simply use pcollection | map_fn, >>>>>>>> instead of pcollection | 'Map' >> map_fn. >>>>>>>> >>>>>>>> See an example here >>>>>>>> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/examples/cookbook/group_with_coder.py#L100-L116 >>>>>>>> >>>>>>>> >>>>>>>> On Sun, Oct 1, 2023 at 9:08 PM Joey Tran <joey.t...@schrodinger.com> >>>>>>>> wrote: >>>>>>>> >>>>>>>>> Hmm, I'm not sure what you mean by "updating pipelines in place". >>>>>>>>> Can you elaborate? >>>>>>>>> >>>>>>>>> I forgot to mention my question is posed from the context of a >>>>>>>>> python SDK user, and afaict, there doesn't seem to be an obvious way >>>>>>>>> to >>>>>>>>> autogenerate names/labels. Hearing that the java SDK supports it >>>>>>>>> makes me >>>>>>>>> wonder if the python SDK could support it as well though... (If so, >>>>>>>>> I'd be >>>>>>>>> happy to do implement it). Currently, it's fairly tedious to have to >>>>>>>>> name >>>>>>>>> every instance of a transform that you might reuse in a pipeline, >>>>>>>>> e.g. when >>>>>>>>> reapplying the same Map on different pcollections. >>>>>>>>> >>>>>>>>> On Sun, Oct 1, 2023 at 8:12 PM Reuven Lax via user < >>>>>>>>> u...@beam.apache.org> wrote: >>>>>>>>> >>>>>>>>>> Are you talking about transform names? The main reason was >>>>>>>>>> because for runners that support updating pipelines in place, the >>>>>>>>>> only way >>>>>>>>>> to do so safely is if the runner can perfectly identify which >>>>>>>>>> transforms in >>>>>>>>>> the new graph match the ones in the old graph. There's no good way >>>>>>>>>> to auto >>>>>>>>>> generate names that will stay stable across updates - even small >>>>>>>>>> changes to >>>>>>>>>> the pipeline might change the order of nodes in the graph, which >>>>>>>>>> could >>>>>>>>>> result in a corrupted update. >>>>>>>>>> >>>>>>>>>> However, if you don't care about update, Beam can auto generate >>>>>>>>>> these names for you! When you call PCollection.apply (if using >>>>>>>>>> BeamJava), >>>>>>>>>> simply omit the name parameter and Beam will auto generate a unique >>>>>>>>>> name >>>>>>>>>> for you. >>>>>>>>>> >>>>>>>>>> Reuven >>>>>>>>>> >>>>>>>>>> On Sat, Sep 30, 2023 at 11:54 AM Joey Tran < >>>>>>>>>> joey.t...@schrodinger.com> wrote: >>>>>>>>>> >>>>>>>>>>> After writing a few pipelines now, I keep getting tripped up >>>>>>>>>>> from accidentally have duplicate labels from using multiple of the >>>>>>>>>>> same >>>>>>>>>>> transforms without labeling them. I figure this must be a common >>>>>>>>>>> complaint, >>>>>>>>>>> so I was just curious, what the rationale behind this design was? >>>>>>>>>>> My naive >>>>>>>>>>> thought off the top of my head is that it'd be more user friendly >>>>>>>>>>> to just >>>>>>>>>>> auto increment duplicate transforms, but I figure I must be missing >>>>>>>>>>> something >>>>>>>>>>> >>>>>>>>>>> Cheers, >>>>>>>>>>> Joey >>>>>>>>>>> >>>>>>>>>>