I agree, it's a decent assumption.

Regards
JB

On 02/17/2018 05:59 PM, Romain Manni-Bucau wrote:
> Assuming a Pipeline.run(); the corresponding sequence:
> 
> WorkerStartFn();
> WorkerEndFn();
> 
> So a single instance of the fn for the full pipeline execution.
> 
> Le 17 févr. 2018 17:42, "Reuven Lax" <re...@google.com
> <mailto:re...@google.com>> a écrit :
> 
>     " and a transform is by design bound to an execution"
> 
>     What do you mean by execution?
> 
>     On Sat, Feb 17, 2018 at 12:50 AM, Romain Manni-Bucau 
> <rmannibu...@gmail.com
>     <mailto:rmannibu...@gmail.com>> wrote:
> 
> 
> 
>         Le 16 févr. 2018 22:41, "Reuven Lax" <re...@google.com
>         <mailto:re...@google.com>> a écrit :
> 
>             Kenn is correct. Allowing Fn reuse across bundles was a major, 
> major
>             performance improvement. Profiling on the old Dataflow SDKs
>             consistently showed Java serialization being the number one
>             performance bottleneck for streaming pipelines, and Beam fixed 
> this.
> 
> 
>         Sorry but this doesnt help me much to understand. Let me try to 
> explain.
>         I read it as "we were slow somehow around serialization so a quick fix
>         was caching".
> 
>         It is not to be picky but i had a lot of remote ejb over rmi super 
> fast
>         setup do java serialization is slower than alternative serialization,
>         right, but doesnt justify caching most of the time.
> 
>         My main interrogation is: isnt beam which is designed to be slow in 
> the
>         way it designed the dofn/transform and therefore serializes way more
>         than it requires - you never care to serialize the full transform and
>         can in 95% do a writeReplace which is light and fast compared to the
>         default.
> 
>         If so the cache is an implementation workaround and not a fix.
> 
>         Hope my view is clearer on it.
> 
> 
> 
>             Romain - can you state precisely what you want? I do think there 
> is
>             still a gap - IMO there's a place for a longer-lived per-fn
>             container; evidence for this is that people still often need to 
> use
>             statics to store things. However I'm not sure if this is what 
> you're
>             looking for.
> 
> 
>         Yes. I build a framework on top of beam and must be able to provide a
>         lifecycle clear and reliable. The bare minimum for any user is
>         start-exec-stop and a transform is by design bound to an execution
>         (stream or batch).
> 
>         Bundles are not an option as explained cause not bound to the 
> execution
>         but an uncontrolled subpart. You can see it as a beam internal until
>         runners unify this definition. And in any case it is closer to a chunk
>         notion than a lifecycle one.
> 
>         So setup and teardown must be symmetric.
> 
>         Note that a dofn instance owns a config so is bound to an execution.
> 
>         This all lead to the nees of a reliable teardown.
> 
>         Caching can be neat bit requires it own api like passivation one of 
> ejbs.
> 
> 
> 
>             Reuven
> 
>             On Fri, Feb 16, 2018 at 1:33 PM, Kenneth Knowles <k...@google.com
>             <mailto:k...@google.com>> wrote:
> 
>                 On Fri, Feb 16, 2018 at 1:00 PM, Romain Manni-Bucau
>                 <rmannibu...@gmail.com <mailto:rmannibu...@gmail.com>> wrote:
> 
>                     The serialization of fn being once per bundle, the perf
>                     impact is only huge if there is a bug somewhere else, even
>                     java serialization is negligeable on big config compared 
> to
>                     any small pipeline (seconds vs minutes).
> 
>                  
>                 Profiling is clear that this is a huge performance impact. One
>                 of the most important backwards-incompatible changes we made 
> for
>                 Beam 2.0.0 was to allow Fn reuse across bundles.
> 
>                 When we used a DoFn only for one bundle, there was no 
> @Teardown
>                 because it has ~no use. You do everything in @FinishBundle. So
>                 for whatever use case you are working on, if your pipeline
>                 performs well enough doing it per bundle, you can put it in
>                 @FinishBundle. Of course it still might not get called because
>                 that is a logical impossibility - you just know that for a 
> given
>                 element the element will be retried if @FinishBundle fails.
> 
>                 If you have cleanup logic that absolutely must get executed,
>                 then you need to build a composite PTransform around it so it
>                 will be retried until cleanup succeeds. In Beam's sinks you 
> can
>                 find many examples.
> 
>                 Kenn
> 
> 
> 
> 

-- 
Jean-Baptiste Onofré
jbono...@apache.org
http://blog.nanthrax.net
Talend - http://www.talend.com

Reply via email to