Another aspect to what Reuven brought up is that it is quite difficult to
dynamically create a DoFn now that it requires annotations or magic
parameters for everything. For example if I have my own DSL and I want to
compile it to a DoFn where the state / parameters of the DoFn depend on the
input being compiled. My proposal for that use case is to have a variant of
the ParDo primitive something like a ParDo.of(DoFnInvoker). The
"parameters" bundle passed to a DoFnInvoker is essentially the same thing
as ye olde ProcessContext.

To be clear: I absolutely support deprecating ProcessContext. I think
addressing the missing use case(s) in other ways is way better than keeping
it around.

Kenn

On Thu, Mar 10, 2022 at 3:26 PM Luke Cwik <lc...@google.com> wrote:

> Another advantage is that once you know the parameters you can replace the
> execution time wrappers with lighter weight wrappers that don't have to
> create and manage many expensive objects (e.g. replace FnApiDoFnRunner with
> MapFnRunner).
>
> On Thu, Mar 10, 2022 at 1:05 PM Robert Bradshaw <rober...@google.com>
> wrote:
>
>> On Thu, Mar 10, 2022 at 11:45 AM Andrew Pilloud <apill...@google.com>
>> wrote:
>>
>>> Thanks for noticing that. It does look like we are missing this use case
>>> for SideInputs. We can add a MultiSideInput interface that allows
>>> fetching arbitrary side inputs, I opened
>>> https://issues.apache.org/jira/browse/BEAM-14085
>>>
>>
>> +1
>>
>> We can, of course, start de-emphasizing it in our docs, etc. for standard
>> uscases sooner rather than later.
>>
>>
>>> On Wed, Mar 9, 2022 at 4:25 PM Reuven Lax <re...@google.com> wrote:
>>>
>>>> One important point - don't forget about systems that use Beam to
>>>> implement other frameworks (e.g. scio, SQL, etc.). These use cases often
>>>> don't statically know the graph a priori, so can't just rely on static
>>>> parameters. e.g. your point about side inputs being solved only works for
>>>> statically known side inputs - a framework would want to be able to specify
>>>> the which side input to read at run time.
>>>>
>>>> On Wed, Mar 9, 2022 at 4:00 PM Andrew Pilloud <apill...@google.com>
>>>> wrote:
>>>>
>>>>> I agree that we are missing pieces needed to depreciate
>>>>> FinishBundleContext, however that interface is much smaller
>>>>> (PipelineOptions plus an extended MultiOutputReceiver) so doesn't suffer
>>>>> from the problems ProcessContext does!
>>>>>
>>>>> Andrew
>>>>>
>>>>> On Wed, Mar 9, 2022 at 3:28 PM Robert Bradshaw <rober...@google.com>
>>>>> wrote:
>>>>>
>>>>>> +1 to deprecating it and removing it from our examples, tutorials,
>>>>>> etc.
>>>>>>
>>>>>> The one thing it could be useful for is forwarding all parameters,
>>>>>> e.g. a wrapper converting a DoFn<V, O> to a DoFn<KV<K, V>>, KV<K, O>>, 
>>>>>> but
>>>>>> that's already pretty cumbersome to do.
>>>>>>
>>>>>> On Wed, Mar 9, 2022 at 2:56 PM Andrew Pilloud <apill...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Hi everyone,
>>>>>>>
>>>>>>> I'm back from baby leave so I thought I'd propose a
>>>>>>> big, controversial change that I've been holding back on: we should mark
>>>>>>> DoFn.ProcessContext deprecated in the Java SDK. The use of 
>>>>>>> ProcessContext
>>>>>>> prevents optimizations based on DoFn metadata as it creates an implicit
>>>>>>> access to both the element and timestamp even if those methods are never
>>>>>>> called.
>>>>>>>
>>>>>>> There are better paths that allow users to get at all the same
>>>>>>> information available through ProcessContext without implicit accesses
>>>>>>> notably writing a ProcessElement with the appropriate type or annotation
>>>>>>> parameters: DoFn.Element (and preferably DoFn.FieldAccess), 
>>>>>>> DoFn.SideInput,
>>>>>>> PaneInfo, DoFn.Timestamp, and DoFn.OutputReceiver. Most of these were 
>>>>>>> added
>>>>>>> in Beam 2.5.0, but the latest (SideInput) was added in 2.16.0. That is 
>>>>>>> to
>>>>>>> say we've supported better options for multiple years.
>>>>>>>
>>>>>>> This is not a proposal to remove ProcessContext (that would be a
>>>>>>> pretty significant breaking change) but I would suggest that we don't 
>>>>>>> add
>>>>>>> any new features to ProcessContext, remove uses from Beam outside of 
>>>>>>> tests
>>>>>>> targeting ProcessContext, and remove it from tutorials.
>>>>>>>
>>>>>>> I opened a JIRA a few months back:
>>>>>>> https://issues.apache.org/jira/browse/BEAM-13057 Is there anything
>>>>>>> I'm missing, anything that would block this change, or any other 
>>>>>>> objections?
>>>>>>>
>>>>>>> Andrew
>>>>>>>
>>>>>>

Reply via email to