You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by "Kenneth Knowles (JIRA)" <ji...@apache.org> on 2016/12/19 04:02:58 UTC

[jira] [Commented] (BEAM-1164) Allow a DoFn to opt in to mutating it's input

    [ https://issues.apache.org/jira/browse/BEAM-1164?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15760074#comment-15760074 ] 

Kenneth Knowles commented on BEAM-1164:
---------------------------------------

I think with the new {{DoFn}} there is a fairly elegant solution here.

Today we write:

{code}
new DoFn<Foo, Baz>() {
  @ProcessElement
  public void processElem(ProcessContext ctx) {
    ... ctx.element() ...
  }
}
{code}

We'd like to allow the user to request only the element, both for clarity and for potential optimization, as in

{code}
new DoFn<Foo, Baz>() {
  @ProcessElement
  public void processElem(Element elem) {
    ... elem.get() ...
  }
}
{code}

where {{Element}} is a distinguished inner class, to avoid repeating verbose input types.

From here, it is a short step to saying that you want a mutable element:

{code}
new DoFn<Foo, Baz>() {
  @ProcessElement
  public void processElem(MutableElement elem) {
    ... elem.get().setBizzle(...) ...
  }
}
{code}

At the level of the "JSON" runner API, we will need to tag the user-defined function with the fact that it intends to mutate its input. The Java SDK will analyze the method signature, as usual, to discern this automatically.

A runner will then be free to decide between disabling optimizations or cloning elements when necessary.

> Allow a DoFn to opt in to mutating it's input
> ---------------------------------------------
>
>                 Key: BEAM-1164
>                 URL: https://issues.apache.org/jira/browse/BEAM-1164
>             Project: Beam
>          Issue Type: Bug
>          Components: beam-model
>            Reporter: Frances Perry
>            Priority: Minor
>
> Runners generally can't tell if a DoFn is mutating inputs, but assuming so by default leads to significant performance implications from unnecessary copying (around sibling fusion, etc). So instead the model prevents mutating inputs, and the Direct Runner validates this behavior. (See: http://beam.incubator.apache.org/contribute/design-principles/#make-efficient-things-easy-rather-than-make-easy-things-efficient) 
> However, if users are processing a small number of large records by making incremental changes (for example, genomics use cases), the cost of immutability requirement can be very large. As a workaround, users sometimes do suboptimal things (fusing ParDos by hand) or undefined things when they expect the immutability requirement is unnecessarily strict (adding no-op coders in places they hope the runner won't be materializing things, mutating things anyway when they don't expect sibling fusion to happen, etc).
> We should consider adding a signal (MutatingDoFn?) that users explicitly opt in to to say their code may mutate inputs. The runner can then use this assumption to either prevent optimizations that would break in the face of this or insert additional copies as needed to allow optimizations to preserve semantics.
> See this related user@ discussion:
> https://lists.apache.org/thread.html/f39689f54147117f3fc54c498eff1a20fa73f1be5b5cad5b6f816fd3@%3Cuser.beam.apache.org%3E



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)