Is there a plan/timeline to resolve the flag? It would be good for this to
be the default, or only, behavior.

The behavior with the flag activated is correct. It may still cause
problems for users who accidentally depended on things outside the Beam
model, so having a flag for them to opt-out until they can migrate might be
good.

Meanwhile there is a larger thread we could start about requesting a
safe-to-mutate copy of things so it only affects performance local to that
DoFn. Users can do this themselves, but if it is metadata on a DoFn it may
be optimized sometimes.

Kenn

On Tue, Nov 24, 2020 at 12:39 AM Jan Lukavský <je...@seznam.cz> wrote:

> Hi Antonio,
>
> the PR is already merged [1]. It will be released as part of the
> upcoming 2.26.0 release [2]. You must activate the feature by using the
> --fasterCopy flag.
>
> Best,
>
>   Jan
>
> [1] https://github.com/apache/beam/pull/13240
>
> [2] https://issues.apache.org/jira/browse/BEAM-11146
>
> On 11/23/20 10:57 PM, Antonio Si wrote:
> > Hi all,
> >
> > Our team recently did a similar experiment and came to a similar
> observation as what Teodor did.
> > The Beam slack channel points me to email thread discussion.
> >
> > It seems like there is a jira issue created and Teodor had a PR. May I
> ask what is the decision on that and if the PR is approved? May I also have
> the link to the jira issue?
> >
> > Much appreciated.
> >
> > Antonio.
> >
> >
> > On 2020/10/30 20:08:32, Kenneth Knowles <k...@apache.org> wrote:
> >> I have not had an easy time following all the discussion points here. It
> >> seems that the main issue is really something that has been true since
> >> before Beam started: " In Beam, the user is not supposed to modify the
> input
> >> collection and if they do, it's undefined behavior." and after Beam
> started
> >> we added features to help: "This is the reason the DirectRunner checks
> for
> >> this, to make sure the users are not relying on it."
> >>
> >> This is all true. A runner can do "chaining" or "fusion" with zero copy
> and
> >> this is allowed. An SDK harness can do the same. If a user mutates an
> input
> >> to a DoFn, or mutates a value after it is output, that is user error. A
> >> runner that eagerly cloning elements is wasting time and we should
> remove
> >> that.
> >>
> >> Kenn
> >>
> >> On Thu, Oct 29, 2020 at 8:03 AM Teodor Spæren <
> teodor_spae...@riseup.net>
> >> wrote:
> >>
> >>> Thanks Jan, this cleared some things up!
> >>>
> >>> Best regards,
> >>> Teodor Spæren
> >>>
> >>> On Thu, Oct 29, 2020 at 02:13:50PM +0100, Jan Lukavský wrote:
> >>>> Hi Teodor,
> >>>>
> >>>> the confusion here maybe comes from the fact, that there are two
> >>>> (logical) representations of an element in PCollection. One
> >>>> representation is the never mutable (most probably serialized in a
> >>>> binary form) form of a PCollection element, where no modifications are
> >>>> possible. Once a PCollection is created (e.g. read from source, or
> >>>> created by a PTransform) it cannot be modified further. The second
> >>>> form is an SDK-dependent representation of each PCollection element in
> >>>> user code. This representation is what UDFs work with. The same source
> >>>> (binary) form of element can have (and will have) different
> >>>> representation in Java SDK and in Python SDK. The Beam model says
> >>>> nothing about mutability of this SDK-dependent form. Nevertheless,
> >>>> even if you modify this element, it has no impact on the source
> >>>> representation. But, it can lead to SDK-dependent errors, when the
> >>>> element is mutated in a way that a runner might not expect.
> >>>>
> >>>> Hope this helps.
> >>>>
> >>>> Jan
> >>>>
> >>>> On 10/29/20 1:58 PM, Teodor Spæren wrote:
> >>>>> Hey!
> >>>>>
> >>>>> Just so I understand this correctly then, what does the following
> >>>>> quote from [1], section 3.2.3 mean:
> >>>>>
> >>>>> A PCollection is immutable. Once created, you cannot add, remove, or
> >>>>> change individual elements. A Beam Transform might process each
> >>>>> element of a PCollection and generate new pipeline data (as a new
> >>>>> PCollection), *but it does not consume or modify the original input
> >>>>> collection.*
> >>>>>
> >>>>> (Don't know what the normal way of highlighting is on mailing lists,
> >>>>> so I just put it between *)
> >>>>>
> >>>>> I read this as meaning that it is the users responsibilty to make
> >>>>> sure that their transformations do not modify the input, but should
> >>>>> I rather read it as meaning the beam runner itself should make sure
> >>>>> the user cannot make such a mistake? I find this reading at odds
> >>>>> with the documentation about the direct runner and it's express
> >>>>> purpose being to make sure users doesn't rely on semantics the beam
> >>>>> model doesn't ensure. And modifying of input arguments being one of
> >>>>> the constraints listed. [2].
> >>>>>
> >>>>> It doesn't change the outcome here, adding an opt out switch, but if
> >>>>> I've missunderstood the quote above, I think this might benefit by
> >>>>> being reworded, to make sure it is communicated that shooting
> >>>>> yourself in the foot is impossible and the direct runner testing of
> >>>>> modifying input should be removed, as there is no point in users
> >>>>> making sure to not modifying the input if all runners guarantee it.
> >>>>>
> >>>>>
> >>>>> Also, I ran the whole Flink test suite with a simple return instead
> >>>>> of the deep copy and all tests passed, so there is no such test in
> >>>>> there. Depending on the reading above, we should add such tests to
> >>>>> all runners.
> >>>>>
> >>>>> Best regards,
> >>>>> Teodor Spæren
> >>>>>
> >>>>> On Thu, Oct 29, 2020 at 10:16:30AM +0100, Maximilian Michels wrote:
> >>>>>>> Ok then we are on the same page, but I disagree with your
> >>>>>>> conclusion. The reason Flink has to do the deep copy is that it
> >>>>>>> doesn't state that the inputs are immutable and should not be
> >>>>>>> changed, and so have to do the deep copy. In Beam, the user is
> >>>>>>> not supposed to modify the input collection and if they do, it's
> >>>>>>> undefined behavior. This is the reason the DirectRunner checks
> >>>>>>> for this, to make sure the users are not relying on it.
> >>>>>> It's not written anywhere that the input cannot be mutated. A
> >>>>>> DirectRunner test is not a proof. Any runner could add a test
> >>>>>> which proves the opposite. In fact we may have one that checks
> >>>>>> copying for Flink.
> >>>>>>
> >>>>>> I prefer safety and correctness over performance because I've seen
> >>>>>> too many cases where users shoot themselves in the foot. We should
> >>>>>> make sure that, by default, the user cannot modify the input
> >>>>>> element. An option to disable that is fine.
> >>>>>>
> >>>>>> -Max
>

Reply via email to