We should start by understanding the goals. If elements are in different
windows can they be out in the same batch? If they have different
timestamps what timestamp should the batch have?

As a composite transform this will likely require a group by key which may
affect performance. Maybe within a dofn is better.

Then it could be some annotation or API that informs the runner. Should
batch sizes be fixed in the annotation (element count or size) or should
the user have some method that lets them decide when to process a batch
based on the contents?

Another thing to think about is whether this should be connected to the
ability to run parts of the bundle in parallel. Maybe each batch is an RPC
and you just want to start an async RPC for each batch. Then in addition to
start the final RPC in finishBundle, you also need to wait for all the RPCs
to complete.
On Tue, Jan 17, 2017, 8:48 AM Etienne Chauchot <echauc...@gmail.com> wrote:

Hi JB,

I meant jira vote but discussion on the ML works also :)

As I understand the need (see stackoverflow links in jira ticket) the
aim is to avoid the user having to code the batching logic in his own
DoFn.processElement() and DoFn.finishBundle() regardless of the bundles.
For example, possible use case is to batch a call to an external service
(for performance).

I was thinking about providing a PTransform that implements the batching
in its own DoFn and that takes user defined functions for customization.

Etienne

Le 17/01/2017 à 17:30, Jean-Baptiste Onofré a écrit :
> Hi
>
> I guess you mean discussion on the mailing list about that, right ?
>
> AFAIR the ide⁣​a is to provide a utility class to deal with
pooling/batching. However not sure it's required as with @StartBundle etc
in DoFn and batching depends of the end user "logic".
>
> Regards
> JB
>
> On Jan 17, 2017, 08:26, at 08:26, Etienne Chauchot <echauc...@gmail.com>
wrote:
>> Hi all,
>>
>> I have started to work on this ticket
>> https://issues.apache.org/jira/browse/BEAM-135
>>
>> As there where no vote since March 18th, is the issue still
>> relevant/needed?
>>
>> Regards,
>>
>> Etienne

Reply via email to