Hi Robert,

I though it would be that case. ParDoLifecycleTest, however, does not currently allow for empty bundles. We have currently worked around this in Flink by avoiding the creation of these bundles, but maybe the test should be modified so that it adheres to the model [1].

Jan

[1] https://github.com/apache/beam/pull/9846

On 10/21/19 6:00 PM, Robert Bradshaw wrote:
Yes, the model allows them.

It also takes less work to avoid them in general (e.g. imagine one
reshuffles N elements to M > N workers. A priori, one would "start" a
bundle and then try to read all data destined for that
worker--postponing this until one knows that the set of data for this
worker could be an optimization (as could not doing so as a form of
speculative execution) but should not be necessary.

- Robert

On Mon, Oct 21, 2019 at 7:03 AM Jan Lukavský <je...@seznam.cz> wrote:
Hi Max,

that is true, but then we have two orthogonal issues:

   a) correctness - if empty bundles are aligned with the model, then
validates runner tests should take that into account

   b) performance - that can be dealt with in separate JIRA issue, if needed

WDYT?

Jan

On 10/21/19 3:22 PM, Maximilian Michels wrote:
Hi Jan,

I think it is aligned with the model to create empty bundles. The
question if course, whether it is preferable to avoid them, since the
Setup/Finish state might be costly, depending on the bundle size and
the type of DoFn used.

Cheers,
Max

On 21.10.19 14:13, Kyle Weaver wrote:
Nevermind, this is discussed on the PR linked.

On Mon, Oct 21, 2019 at 2:11 PM Kyle Weaver <kcwea...@google.com
<mailto:kcwea...@google.com>> wrote:

     Do you know why an empty bundle might be created?

     On Mon, Oct 21, 2019 at 1:42 PM Jan Lukavský <je...@seznam.cz
     <mailto:je...@seznam.cz>> wrote:

         Hi,

         when debugging a flaky ParDoLifecycleTest in FlinkRunner, I have
         found a
         situation, where Flink might create empty bundle - i.e. call
         @StartBundle immediately followed by @FinishBundle, with no
         elements
         inside the bundle. That is what breaks the ParDoLifecycleTest,
         because
         the test explicitly assumes, that the sequence of lifecycle
methods
         should be StartBundle -> Process Element -> Finish Bundle. It is
         easy to
         modify the test to accept situation of StartBundle ->
         FinishBundle with
         no elements ([1]), but the question is, is this allowed by the
         model? I
         think there is no reason not to be, but I'd like to be sure.

         Thanks,

            Jan

         [1] https://github.com/apache/beam/pull/9841

Reply via email to