On Tue, Aug 25, 2020 at 8:20 PM Ke Wu <[email protected]> wrote: > Thank you all for the reply. One last question, I noticed that > ParDoTest$StateTests > > testValueStateSameId > <https://github.com/apache/beam/blob/c7e8c6bdb819b0d32f8727036a09c227529d01d0/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java#L2118> > seems > to be testing exact this case, however, the first ParDo intentionally > changed the key of inputs thus the subsequent ParDo would never share the > same state cell anyway. Is this expected or do we want to actually want to > test that same state id in different DoFn(s) is actually completely > separate? >
Good catch. You are right that the test is wrong. The two DoFns should have the same keys, windows, and stateids. Would you care to fix it? Kenn > > On Aug 21, 2020, at 4:50 PM, Robert Bradshaw <[email protected]> wrote: > > We should be using PTransform Labels (aka Names), not ids, for naming > state. This is why the names must line up when doing, for example, a > Dataflow update operation with Stateful DoFns. > > (And, yes, if the user does not specify the transform name, and it is > autogenerated differently, this will be an error. This is why we throw > exceptions in the SDK if a name is re-used rather than just appending > a counter or similar.) > > > On Fri, Aug 21, 2020 at 4:12 PM Ke Wu <[email protected]> wrote: > > > If user does not explicitly specify transform name, in which case a > autogenerated name will be used when generating the unique id, does it > mean, the id could change when the pipeline changes, such as adding extra > transforms etc? > > On Aug 21, 2020, at 11:43 AM, Luke Cwik <[email protected]> wrote: > > The DoFn is associated with a PTransform and in the pipeline proto there > is a unique id associated with each PTransform. You can use that to > generate a composite key (ptransformid, stateid) which will be unique > within the pipeline. > > On Fri, Aug 21, 2020 at 11:26 AM Ke Wu <[email protected]> wrote: > > > Thank you Reuven for the confirmation. Do you know what is the recommended > way for underlying runners to distinguish same state id in different > DoFn(s)? > > On Aug 21, 2020, at 10:27 AM, Reuven Lax <[email protected]> wrote: > > StateId is scoped to the DoFn. You can use the same string in different > DoFns for completely different states. > > On Fri, Aug 21, 2020 at 10:21 AM Ke Wu <[email protected]> wrote: > > > Hello everyone, > > After reading through Stateful processing with Apache Beam and > DoFn.StateId, I understand that each state id must be unique and must be > the same type at least in the same DoFn, however, it does not explicitly > mention whether or not it is expected and supported that the same state id > to be declared in different DoFn(s). If Yes, is the state supposed to be a > shared state or is supposed to completed separate, therefore it could even > be different types. If No, it seems that the validation in Beam SDK only > validates uniqueness in the same DoFn. > > Thanks, > Ke > > > > > >
