On Wed, Apr 10, 2019 at 8:18 PM Ahmet Altay <al...@google.com> wrote:

>
>
> On Wed, Apr 10, 2019 at 7:59 PM Kenneth Knowles <k...@apache.org> wrote:
>
>> TL;DR I like the simple approach better than the ServiceLoader solution
>> when a particular DoFn depends on the result. The ServiceLoader solution
>> fits when it is somewhat independent of a particular DoFn (I'm not sure the
>> use case(s)).
>>
>> On Wed, Apr 10, 2019 at 4:10 PM Brian Hulette <bhule...@google.com>
>> wrote:
>>
>>> - Each DoFn that depends on that initialization needs to include the
>>> same initialization
>>>
>>
>> What if a DoFn that depends on the initialization is used in a new
>> context? Then it is relying on initialization done elsewhere, and it will
>> break or, worse, give wrong results. So I think this bullet point is a
>> feature, not a bug. And if the initialization is built as a static method
>> of some third class, referenced by all the DoFns that need it, it is a
>> one-liner to declare the dependency explicitly.
>>
>>
>>> - There is no way for users to know which workers executed a particular
>>> DoFn - users could have workers with different configurations
>>>
>>
>> What is a worker? j/k. Each runner has different notions of what a worker
>> is, including the Java SDK Harness. But they all do require one or more
>> JVMs. It is true that you can't easily predict which DoFn classes are
>> loaded on a particular JVM. This bullet is a strong case against
>> initialization at a distance. I think your proposed solution and also the
>> simple static block approach avoid this pitfall, so all is good.
>>
>> You could perhaps argue that these are actually good things - we only run
>>> the initialization when it's needed - but it could also lead to confusing
>>> behavior.
>>>
>>
>> FWIW my argument above is not about only running when needed. The
>> opposite - it is about being certain it is run when needed.
>>
>>
>>> So I'd like to a propose an addition to the Java SDK that provides hooks
>>> for JVM initialization that is guaranteed to execute once across all worker
>>> workers. I've written up a PR [1] that implements this. It adds a service
>>> interface, BeamWorkerInitializer, that users can implement to define some
>>> initialization, and modifies workers (currently just the portable worker
>>> and the dataflow worker) to find and execute these implementations using
>>> ServiceLoader. BeamWorkerInitializer has two methods that can be overriden:
>>> onStartup, which workers run immediately after starting, and
>>> beforeProcessing, which workers run after initializing things like logging,
>>> but before beginning to process data.
>>>
>>> Since this is a pretty fundamental change I wanted to have a quick
>>> discussion here before merging, in case there are any comments or concerns.
>>>
>>
>> FWIW (again) I have no objection to the general idea and don't have any
>> problem with making such a fundamental change. I actually think your change
>> is probably useful. But if a particular DoFn depends on the JVM being
>> configured a certain way, a static block in that DoFn class seems more
>> readable and reliable.
>>
>> Are there use cases for more generic JVM initialization that, presumably,
>> a user would want to affect all their DoFns?
>>
>
> A few things I can recall from recent user interactions are a need for
> setting a custom ssl providers, time zone rules providers. Users would want
> such settings to apply for all their dofns in a pipeline.
>

This makes sense. Another perspective is whether the
initialization/configuration might be orthogonal to the DoFns in the
pipeline. These seem to fit that description.

Kenn


>
>
>>
>> Kenn
>>
>>
>>> Thanks!
>>> Brian
>>>
>>> [1] https://github.com/apache/beam/pull/8104
>>>
>>

Reply via email to