You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@beam.apache.org by Eddy G <ka...@gmail.com> on 2021/06/14 11:39:36 UTC

GroupIntoShards not sending bytes further when dealing with huge amount of data

As seen in this image https://imgur.com/a/wrZET97, I'm trying to deal with late data (intentionally stopped my consumer so data has been accumulating for several days now). Now, with the following Window... I'm using Beam 2.27 and Flink 1.12.

                            Window.into(FixedWindows.of(Duration.standardMinutes(10)))

And several parsing stages after, once it's time to write within the ParquetIO stage...

                            FileIO
                                .<String, MyClass>writeDynamic()
                                .by(...)
                                .via(...)
                                .to(...)
                                .withNaming(...)
                                .withDestinationCoder(StringUtf8Coder.of())
                                .withNumShards(options.getNumShards())

it won't send bytes across all stages so no data is being written, still it accumulates in the first stage seen in the image and won't go further than that.

Any reason why this may be happening? Wrong windowing strategy?

Re: GroupIntoShards not sending bytes further when dealing with huge amount of data

Posted by Evan Galpin <ev...@gmail.com>.
I believe that by default windows will only trigger one time [1]. This has
definitely caught me by surprise before.

I think that default strategy might fine for a batch pipeline,
but  typically does not for streaming (which I assume you’re using because
you mentioned Flink).

I believe you’ll want to add a non-default triggering mechanism to the
window strategy that you mentioned. I would recommend reading through the
triggering docs[2] for background. The Repeatedly.forever[3] function may
work for your use case. Something like:

Window.into(FixedWindows.of(Duration.standardMinutes(10)))

.triggering(Repeatedly.forever(AfterWatermark.pastEndOfWindow()))
                    .withAllowedLateness(Duration.ZERO)
                    .discardingFiredPanes();


[1]
https://beam.apache.org/documentation/programming-guide/#default-trigger
[2]
https://beam.apache.org/documentation/programming-guide/#triggers
[3]
https://beam.apache.org/releases/javadoc/2.29.0/org/apache/beam/sdk/transforms/windowing/Repeatedly.html

On Mon, Jun 14, 2021 at 07:39 Eddy G <ka...@gmail.com> wrote:

> As seen in this image https://imgur.com/a/wrZET97, I'm trying to deal
> with late data (intentionally stopped my consumer so data has been
> accumulating for several days now). Now, with the following Window... I'm
> using Beam 2.27 and Flink 1.12.
>
>
> Window.into(FixedWindows.of(Duration.standardMinutes(10)))
>
> And several parsing stages after, once it's time to write within the
> ParquetIO stage...
>
>                             FileIO
>                                 .<String, MyClass>writeDynamic()
>                                 .by(...)
>                                 .via(...)
>                                 .to(...)
>                                 .withNaming(...)
>                                 .withDestinationCoder(StringUtf8Coder.of())
>                                 .withNumShards(options.getNumShards())
>
> it won't send bytes across all stages so no data is being written, still
> it accumulates in the first stage seen in the image and won't go further
> than that.
>
> Any reason why this may be happening? Wrong windowing strategy?
>

Re: GroupIntoShards not sending bytes further when dealing with huge amount of data

Posted by Evan Galpin <ev...@gmail.com>.
There have been varied reports of slowness loosely attributed to SDF
default wrapper change from 2.25.0.  Ex

https://lists.apache.org/thread.html/re6b0941a8b4951293a0327ce9b25e607cafd6e45b69783f65290edee%40%3Cdev.beam.apache.org%3E

https://issues.apache.org/jira/plugins/servlet/mobile#issue/BEAM-10670/comment/17316858

http://mail-archives.apache.org/mod_mbox/beam-dev/202105.mbox/%3cCAE7Uba_V0vfL9cK7n06N2Zf6e+XcebdiRceZ7YFtLfWuVcHHMQ@mail.gmail.com%3e



On Mon, Jun 14, 2021 at 10:11 Eddy G <ka...@gmail.com> wrote:

> Hi Jan,
>
> I've added --experiments=use_deprecated_read and it seems to work
> flawlessly (with my current Window and the one proposed by Evan).
>
> Why is this? Do Splittable DoFn now break current implementations? Are
> there any posts of possible breaking changes?
>
> On 2021/06/14 13:19:39, Jan Lukavský <je...@seznam.cz> wrote:
> > Hi Eddy,
> >
> > answers inline.
> >
> > On 6/14/21 3:05 PM, Eddy G wrote:
> > > Hi Jan,
> > >
> > > Thanks for replying so fast!
> > >
> > > Regarding your questions,
> > >
> > > - "Does your data get buffered in a state?"
> > > Yes, I do have a state within a stage prior ParquetIO writing together
> with a Timer with PROCESSING_TIME.
> > >
> > > The stage which contains the state does send bytes to the next one
> which is the ParquetIO writing. Seems the @OnTimer doesn't get triggered
> and it's not clearing the state. This however does work under normal
> circumstances without having too much data queued waiting to be processed.
> > OK, this suggests, that the watermark is for some reason "stuck". If you
> > checkpoints enabled, you should see the size of the checkpoint to grow
> > over time.
> > >
> > > - "Do you see watermark being updated in your Flink WebUI?"
> > > The stages that do have a watermark don't get updated. The same
> watermark value has been constant since the pipeline started.
> > >
> > > If no lateness is set, any late data should be admitted right?
> > If no lateness is set, it means allowed lateness of Duration.ZERO, which
> > means that data that arrive after end-of-window will be dropped.
> > >
> > > Regarding 'droppedDueToLateness' metric, can't see it exposed
> anywhere, neither in Flink UI or Prometheus. I've seen it in Dataflow but
> seems to be a Dataflow specific metric right?
> > Should not be Dataflow specific. But if you don't see it, it means it
> > could be zero. So, we can rule this out.
> > >
> > > We're using KinesisIO for reading messages.
> > Kinesis uses UnboundedSource, which is expended to SDF starting from
> > Beam 2.25.0. The flag should change that as well. Can you try the
> > --experiments=use_deprecated_read and see if you Pipeline DAG changes
> > (should not contain Impulse transform at the beginning) and if it solves
> > your issues?
> > >
> > > On 2021/06/14 12:48:58, Jan Lukavský <je...@seznam.cz> wrote:
> > >> Hi Eddy,
> > >>
> > >> does your data get buffered in a state - e.g. does the size of the
> state
> > >> grow over time? Do you see watermark being updated in your Flink
> WebUI?
> > >> When a stateful operation (and GroupByKey is a stateful operation)
> does
> > >> not output any data, the first place to look at is if watermark
> > >> correctly progresses. If it does not progress, then the input data
> must
> > >> be buffered in state and the size of the state should grow over time.
> If
> > >> it progresses, then it might be the case, that the data is too late
> > >> after the watermark (the watermark estimator might need tuning) and
> the
> > >> data gets dropped (note you don't set any allowed lateness, which
> > >> _might_ cause issues). You could see if your pipeline drops data in
> > >> "droppedDueToLateness" metric. The size of you state would not grow
> much
> > >> in that situation.
> > >>
> > >> Another hint - If you use KafkaIO, try to disable SDF wrapper for it
> > >> using "--experiments=use_deprecated_read" on command line (which you
> > >> then must pass to PipelineOptionsFactory). There is some suspicion
> that
> > >> SDF wrapper for Kafka might not work as expected in certain situations
> > >> with Flink.
> > >>
> > >> Please feel free to share any results,
> > >>
> > >>     Jan
> > >>
> > >> On 6/14/21 1:39 PM, Eddy G wrote:
> > >>> As seen in this image https://imgur.com/a/wrZET97, I'm trying to
> deal with late data (intentionally stopped my consumer so data has been
> accumulating for several days now). Now, with the following Window... I'm
> using Beam 2.27 and Flink 1.12.
> > >>>
> > >>>
>  Window.into(FixedWindows.of(Duration.standardMinutes(10)))
> > >>>
> > >>> And several parsing stages after, once it's time to write within the
> ParquetIO stage...
> > >>>
> > >>>                               FileIO
> > >>>                                   .<String, MyClass>writeDynamic()
> > >>>                                   .by(...)
> > >>>                                   .via(...)
> > >>>                                   .to(...)
> > >>>                                   .withNaming(...)
> > >>>
>  .withDestinationCoder(StringUtf8Coder.of())
> > >>>
>  .withNumShards(options.getNumShards())
> > >>>
> > >>> it won't send bytes across all stages so no data is being written,
> still it accumulates in the first stage seen in the image and won't go
> further than that.
> > >>>
> > >>> Any reason why this may be happening? Wrong windowing strategy?
> >
>

Re: GroupIntoShards not sending bytes further when dealing with huge amount of data

Posted by Evan Galpin <ev...@gmail.com>.
I wanted to circle back here and state that using release 2.31.0, with _or_
without  --experiments=use_deprecated_read, seems to resolve the slowness
in my case. It's still on my radar to get a minimal pipeline extracted from
my previously problematic pipeline so as to hopefully aid in debugging
efforts.

Thanks,
Evan

On Mon, Jun 14, 2021 at 8:39 PM Robert Bradshaw <ro...@google.com> wrote:

> Awesome, thanks!
>
> On Mon, Jun 14, 2021 at 5:36 PM Evan Galpin <ev...@gmail.com> wrote:
> >
> > I’ll try to create something as small as possible from the pipeline I
> mentioned 👍 I should have time this week to do so.
> >
> > Thanks,
> > Evan
> >
> > On Mon, Jun 14, 2021 at 18:09 Robert Bradshaw <ro...@google.com>
> wrote:
> >>
> >> Is it possible to post the code? (Or the code of a similar, but
> >> minimal, pipeline that exhibits the same issues?)
> >>
> >> On Mon, Jun 14, 2021 at 2:15 PM Evan Galpin <ev...@gmail.com>
> wrote:
> >> >
> >> > @robert I have a pipeline which consistently shows a major slowdown
> (10 seconds Vs 10 minutes) between version <=2.23.0 and >=2.25.0 that can
> be boiled down to:
> >> >
> >> > - Read GCS file patterns from PubSub
> >> > - Window into Fixed windows (repeating every 15 seconds)
> >> > - Deduplicate/distinct (have tried both)
> >> > - Read GCS blobs via patterns from the first step
> >> > - Write file contents to sink
> >> >
> >> > It doesn't seem to matter if there are 0 messages in a subscription
> or 50k messages at startup. The rate of new messages however is very low.
> Not sure if those are helpful details, let me know if there's anything else
> specific which would help.
> >> >
> >> > On Mon, Jun 14, 2021 at 12:44 PM Robert Bradshaw <ro...@google.com>
> wrote:
> >> >>
> >> >> +1, we'd really like to get to the bottom of this, so clear
> >> >> instructions on a pipeline/conditions that can reproduce it would be
> >> >> great.
> >> >>
> >> >> On Mon, Jun 14, 2021 at 7:34 AM Jan Lukavský <je...@seznam.cz>
> wrote:
> >> >> >
> >> >> > Hi Eddy,
> >> >> >
> >> >> > you are probably hitting a not-yet discovered bug in SDF
> implementation in FlinkRunner that (under some currently unknown
> conditions) seems to stop advancing the watermark. This has been observed
> in one other instance (that I'm aware of). I think we don't yet have a
> tracking JIRA for that, would you mind filling it? It would be awesome if
> you could include estimations of messages per sec throughput that causes
> the issue in your case.
> >> >> >
> >> >> > +Tobias Kaymak
> >> >> >
> >> >> > Tobias, could you please confirm that the case you had with Flink
> stopping progressing watermark resembled this one?
> >> >> >
> >> >> > Thanks.
> >> >> >
> >> >> >  Jan
> >> >> >
> >> >> > On 6/14/21 4:11 PM, Eddy G wrote:
> >> >> >
> >> >> > Hi Jan,
> >> >> >
> >> >> > I've added --experiments=use_deprecated_read and it seems to work
> flawlessly (with my current Window and the one proposed by Evan).
> >> >> >
> >> >> > Why is this? Do Splittable DoFn now break current implementations?
> Are there any posts of possible breaking changes?
> >> >> >
> >> >> > On 2021/06/14 13:19:39, Jan Lukavský <je...@seznam.cz> wrote:
> >> >> >
> >> >> > Hi Eddy,
> >> >> >
> >> >> > answers inline.
> >> >> >
> >> >> > On 6/14/21 3:05 PM, Eddy G wrote:
> >> >> >
> >> >> > Hi Jan,
> >> >> >
> >> >> > Thanks for replying so fast!
> >> >> >
> >> >> > Regarding your questions,
> >> >> >
> >> >> > - "Does your data get buffered in a state?"
> >> >> > Yes, I do have a state within a stage prior ParquetIO writing
> together with a Timer with PROCESSING_TIME.
> >> >> >
> >> >> > The stage which contains the state does send bytes to the next one
> which is the ParquetIO writing. Seems the @OnTimer doesn't get triggered
> and it's not clearing the state. This however does work under normal
> circumstances without having too much data queued waiting to be processed.
> >> >> >
> >> >> > OK, this suggests, that the watermark is for some reason "stuck".
> If you
> >> >> > checkpoints enabled, you should see the size of the checkpoint to
> grow
> >> >> > over time.
> >> >> >
> >> >> > - "Do you see watermark being updated in your Flink WebUI?"
> >> >> > The stages that do have a watermark don't get updated. The same
> watermark value has been constant since the pipeline started.
> >> >> >
> >> >> > If no lateness is set, any late data should be admitted right?
> >> >> >
> >> >> > If no lateness is set, it means allowed lateness of Duration.ZERO,
> which
> >> >> > means that data that arrive after end-of-window will be dropped.
> >> >> >
> >> >> > Regarding 'droppedDueToLateness' metric, can't see it exposed
> anywhere, neither in Flink UI or Prometheus. I've seen it in Dataflow but
> seems to be a Dataflow specific metric right?
> >> >> >
> >> >> > Should not be Dataflow specific. But if you don't see it, it means
> it
> >> >> > could be zero. So, we can rule this out.
> >> >> >
> >> >> > We're using KinesisIO for reading messages.
> >> >> >
> >> >> > Kinesis uses UnboundedSource, which is expended to SDF starting
> from
> >> >> > Beam 2.25.0. The flag should change that as well. Can you try the
> >> >> > --experiments=use_deprecated_read and see if you Pipeline DAG
> changes
> >> >> > (should not contain Impulse transform at the beginning) and if it
> solves
> >> >> > your issues?
> >> >> >
> >> >> > On 2021/06/14 12:48:58, Jan Lukavský <je...@seznam.cz> wrote:
> >> >> >
> >> >> > Hi Eddy,
> >> >> >
> >> >> > does your data get buffered in a state - e.g. does the size of the
> state
> >> >> > grow over time? Do you see watermark being updated in your Flink
> WebUI?
> >> >> > When a stateful operation (and GroupByKey is a stateful operation)
> does
> >> >> > not output any data, the first place to look at is if watermark
> >> >> > correctly progresses. If it does not progress, then the input data
> must
> >> >> > be buffered in state and the size of the state should grow over
> time. If
> >> >> > it progresses, then it might be the case, that the data is too late
> >> >> > after the watermark (the watermark estimator might need tuning)
> and the
> >> >> > data gets dropped (note you don't set any allowed lateness, which
> >> >> > _might_ cause issues). You could see if your pipeline drops data in
> >> >> > "droppedDueToLateness" metric. The size of you state would not
> grow much
> >> >> > in that situation.
> >> >> >
> >> >> > Another hint - If you use KafkaIO, try to disable SDF wrapper for
> it
> >> >> > using "--experiments=use_deprecated_read" on command line (which
> you
> >> >> > then must pass to PipelineOptionsFactory). There is some suspicion
> that
> >> >> > SDF wrapper for Kafka might not work as expected in certain
> situations
> >> >> > with Flink.
> >> >> >
> >> >> > Please feel free to share any results,
> >> >> >
> >> >> >     Jan
> >> >> >
> >> >> > On 6/14/21 1:39 PM, Eddy G wrote:
> >> >> >
> >> >> > As seen in this image https://imgur.com/a/wrZET97, I'm trying to
> deal with late data (intentionally stopped my consumer so data has been
> accumulating for several days now). Now, with the following Window... I'm
> using Beam 2.27 and Flink 1.12.
> >> >> >
> >> >> >
>  Window.into(FixedWindows.of(Duration.standardMinutes(10)))
> >> >> >
> >> >> > And several parsing stages after, once it's time to write within
> the ParquetIO stage...
> >> >> >
> >> >> >                               FileIO
> >> >> >                                   .<String, MyClass>writeDynamic()
> >> >> >                                   .by(...)
> >> >> >                                   .via(...)
> >> >> >                                   .to(...)
> >> >> >                                   .withNaming(...)
> >> >> >
>  .withDestinationCoder(StringUtf8Coder.of())
> >> >> >
>  .withNumShards(options.getNumShards())
> >> >> >
> >> >> > it won't send bytes across all stages so no data is being written,
> still it accumulates in the first stage seen in the image and won't go
> further than that.
> >> >> >
> >> >> > Any reason why this may be happening? Wrong windowing strategy?
>

Re: GroupIntoShards not sending bytes further when dealing with huge amount of data

Posted by Robert Bradshaw <ro...@google.com>.
Awesome, thanks!

On Mon, Jun 14, 2021 at 5:36 PM Evan Galpin <ev...@gmail.com> wrote:
>
> I’ll try to create something as small as possible from the pipeline I mentioned 👍 I should have time this week to do so.
>
> Thanks,
> Evan
>
> On Mon, Jun 14, 2021 at 18:09 Robert Bradshaw <ro...@google.com> wrote:
>>
>> Is it possible to post the code? (Or the code of a similar, but
>> minimal, pipeline that exhibits the same issues?)
>>
>> On Mon, Jun 14, 2021 at 2:15 PM Evan Galpin <ev...@gmail.com> wrote:
>> >
>> > @robert I have a pipeline which consistently shows a major slowdown (10 seconds Vs 10 minutes) between version <=2.23.0 and >=2.25.0 that can be boiled down to:
>> >
>> > - Read GCS file patterns from PubSub
>> > - Window into Fixed windows (repeating every 15 seconds)
>> > - Deduplicate/distinct (have tried both)
>> > - Read GCS blobs via patterns from the first step
>> > - Write file contents to sink
>> >
>> > It doesn't seem to matter if there are 0 messages in a subscription or 50k messages at startup. The rate of new messages however is very low. Not sure if those are helpful details, let me know if there's anything else specific which would help.
>> >
>> > On Mon, Jun 14, 2021 at 12:44 PM Robert Bradshaw <ro...@google.com> wrote:
>> >>
>> >> +1, we'd really like to get to the bottom of this, so clear
>> >> instructions on a pipeline/conditions that can reproduce it would be
>> >> great.
>> >>
>> >> On Mon, Jun 14, 2021 at 7:34 AM Jan Lukavský <je...@seznam.cz> wrote:
>> >> >
>> >> > Hi Eddy,
>> >> >
>> >> > you are probably hitting a not-yet discovered bug in SDF implementation in FlinkRunner that (under some currently unknown conditions) seems to stop advancing the watermark. This has been observed in one other instance (that I'm aware of). I think we don't yet have a tracking JIRA for that, would you mind filling it? It would be awesome if you could include estimations of messages per sec throughput that causes the issue in your case.
>> >> >
>> >> > +Tobias Kaymak
>> >> >
>> >> > Tobias, could you please confirm that the case you had with Flink stopping progressing watermark resembled this one?
>> >> >
>> >> > Thanks.
>> >> >
>> >> >  Jan
>> >> >
>> >> > On 6/14/21 4:11 PM, Eddy G wrote:
>> >> >
>> >> > Hi Jan,
>> >> >
>> >> > I've added --experiments=use_deprecated_read and it seems to work flawlessly (with my current Window and the one proposed by Evan).
>> >> >
>> >> > Why is this? Do Splittable DoFn now break current implementations? Are there any posts of possible breaking changes?
>> >> >
>> >> > On 2021/06/14 13:19:39, Jan Lukavský <je...@seznam.cz> wrote:
>> >> >
>> >> > Hi Eddy,
>> >> >
>> >> > answers inline.
>> >> >
>> >> > On 6/14/21 3:05 PM, Eddy G wrote:
>> >> >
>> >> > Hi Jan,
>> >> >
>> >> > Thanks for replying so fast!
>> >> >
>> >> > Regarding your questions,
>> >> >
>> >> > - "Does your data get buffered in a state?"
>> >> > Yes, I do have a state within a stage prior ParquetIO writing together with a Timer with PROCESSING_TIME.
>> >> >
>> >> > The stage which contains the state does send bytes to the next one which is the ParquetIO writing. Seems the @OnTimer doesn't get triggered and it's not clearing the state. This however does work under normal circumstances without having too much data queued waiting to be processed.
>> >> >
>> >> > OK, this suggests, that the watermark is for some reason "stuck". If you
>> >> > checkpoints enabled, you should see the size of the checkpoint to grow
>> >> > over time.
>> >> >
>> >> > - "Do you see watermark being updated in your Flink WebUI?"
>> >> > The stages that do have a watermark don't get updated. The same watermark value has been constant since the pipeline started.
>> >> >
>> >> > If no lateness is set, any late data should be admitted right?
>> >> >
>> >> > If no lateness is set, it means allowed lateness of Duration.ZERO, which
>> >> > means that data that arrive after end-of-window will be dropped.
>> >> >
>> >> > Regarding 'droppedDueToLateness' metric, can't see it exposed anywhere, neither in Flink UI or Prometheus. I've seen it in Dataflow but seems to be a Dataflow specific metric right?
>> >> >
>> >> > Should not be Dataflow specific. But if you don't see it, it means it
>> >> > could be zero. So, we can rule this out.
>> >> >
>> >> > We're using KinesisIO for reading messages.
>> >> >
>> >> > Kinesis uses UnboundedSource, which is expended to SDF starting from
>> >> > Beam 2.25.0. The flag should change that as well. Can you try the
>> >> > --experiments=use_deprecated_read and see if you Pipeline DAG changes
>> >> > (should not contain Impulse transform at the beginning) and if it solves
>> >> > your issues?
>> >> >
>> >> > On 2021/06/14 12:48:58, Jan Lukavský <je...@seznam.cz> wrote:
>> >> >
>> >> > Hi Eddy,
>> >> >
>> >> > does your data get buffered in a state - e.g. does the size of the state
>> >> > grow over time? Do you see watermark being updated in your Flink WebUI?
>> >> > When a stateful operation (and GroupByKey is a stateful operation) does
>> >> > not output any data, the first place to look at is if watermark
>> >> > correctly progresses. If it does not progress, then the input data must
>> >> > be buffered in state and the size of the state should grow over time. If
>> >> > it progresses, then it might be the case, that the data is too late
>> >> > after the watermark (the watermark estimator might need tuning) and the
>> >> > data gets dropped (note you don't set any allowed lateness, which
>> >> > _might_ cause issues). You could see if your pipeline drops data in
>> >> > "droppedDueToLateness" metric. The size of you state would not grow much
>> >> > in that situation.
>> >> >
>> >> > Another hint - If you use KafkaIO, try to disable SDF wrapper for it
>> >> > using "--experiments=use_deprecated_read" on command line (which you
>> >> > then must pass to PipelineOptionsFactory). There is some suspicion that
>> >> > SDF wrapper for Kafka might not work as expected in certain situations
>> >> > with Flink.
>> >> >
>> >> > Please feel free to share any results,
>> >> >
>> >> >     Jan
>> >> >
>> >> > On 6/14/21 1:39 PM, Eddy G wrote:
>> >> >
>> >> > As seen in this image https://imgur.com/a/wrZET97, I'm trying to deal with late data (intentionally stopped my consumer so data has been accumulating for several days now). Now, with the following Window... I'm using Beam 2.27 and Flink 1.12.
>> >> >
>> >> >                               Window.into(FixedWindows.of(Duration.standardMinutes(10)))
>> >> >
>> >> > And several parsing stages after, once it's time to write within the ParquetIO stage...
>> >> >
>> >> >                               FileIO
>> >> >                                   .<String, MyClass>writeDynamic()
>> >> >                                   .by(...)
>> >> >                                   .via(...)
>> >> >                                   .to(...)
>> >> >                                   .withNaming(...)
>> >> >                                   .withDestinationCoder(StringUtf8Coder.of())
>> >> >                                   .withNumShards(options.getNumShards())
>> >> >
>> >> > it won't send bytes across all stages so no data is being written, still it accumulates in the first stage seen in the image and won't go further than that.
>> >> >
>> >> > Any reason why this may be happening? Wrong windowing strategy?

Re: GroupIntoShards not sending bytes further when dealing with huge amount of data

Posted by Evan Galpin <ev...@gmail.com>.
I’ll try to create something as small as possible from the pipeline I
mentioned 👍 I should have time this week to do so.

Thanks,
Evan

On Mon, Jun 14, 2021 at 18:09 Robert Bradshaw <ro...@google.com> wrote:

> Is it possible to post the code? (Or the code of a similar, but
> minimal, pipeline that exhibits the same issues?)
>
> On Mon, Jun 14, 2021 at 2:15 PM Evan Galpin <ev...@gmail.com> wrote:
> >
> > @robert I have a pipeline which consistently shows a major slowdown (10
> seconds Vs 10 minutes) between version <=2.23.0 and >=2.25.0 that can be
> boiled down to:
> >
> > - Read GCS file patterns from PubSub
> > - Window into Fixed windows (repeating every 15 seconds)
> > - Deduplicate/distinct (have tried both)
> > - Read GCS blobs via patterns from the first step
> > - Write file contents to sink
> >
> > It doesn't seem to matter if there are 0 messages in a subscription or
> 50k messages at startup. The rate of new messages however is very low. Not
> sure if those are helpful details, let me know if there's anything else
> specific which would help.
> >
> > On Mon, Jun 14, 2021 at 12:44 PM Robert Bradshaw <ro...@google.com>
> wrote:
> >>
> >> +1, we'd really like to get to the bottom of this, so clear
> >> instructions on a pipeline/conditions that can reproduce it would be
> >> great.
> >>
> >> On Mon, Jun 14, 2021 at 7:34 AM Jan Lukavský <je...@seznam.cz> wrote:
> >> >
> >> > Hi Eddy,
> >> >
> >> > you are probably hitting a not-yet discovered bug in SDF
> implementation in FlinkRunner that (under some currently unknown
> conditions) seems to stop advancing the watermark. This has been observed
> in one other instance (that I'm aware of). I think we don't yet have a
> tracking JIRA for that, would you mind filling it? It would be awesome if
> you could include estimations of messages per sec throughput that causes
> the issue in your case.
> >> >
> >> > +Tobias Kaymak
> >> >
> >> > Tobias, could you please confirm that the case you had with Flink
> stopping progressing watermark resembled this one?
> >> >
> >> > Thanks.
> >> >
> >> >  Jan
> >> >
> >> > On 6/14/21 4:11 PM, Eddy G wrote:
> >> >
> >> > Hi Jan,
> >> >
> >> > I've added --experiments=use_deprecated_read and it seems to work
> flawlessly (with my current Window and the one proposed by Evan).
> >> >
> >> > Why is this? Do Splittable DoFn now break current implementations?
> Are there any posts of possible breaking changes?
> >> >
> >> > On 2021/06/14 13:19:39, Jan Lukavský <je...@seznam.cz> wrote:
> >> >
> >> > Hi Eddy,
> >> >
> >> > answers inline.
> >> >
> >> > On 6/14/21 3:05 PM, Eddy G wrote:
> >> >
> >> > Hi Jan,
> >> >
> >> > Thanks for replying so fast!
> >> >
> >> > Regarding your questions,
> >> >
> >> > - "Does your data get buffered in a state?"
> >> > Yes, I do have a state within a stage prior ParquetIO writing
> together with a Timer with PROCESSING_TIME.
> >> >
> >> > The stage which contains the state does send bytes to the next one
> which is the ParquetIO writing. Seems the @OnTimer doesn't get triggered
> and it's not clearing the state. This however does work under normal
> circumstances without having too much data queued waiting to be processed.
> >> >
> >> > OK, this suggests, that the watermark is for some reason "stuck". If
> you
> >> > checkpoints enabled, you should see the size of the checkpoint to grow
> >> > over time.
> >> >
> >> > - "Do you see watermark being updated in your Flink WebUI?"
> >> > The stages that do have a watermark don't get updated. The same
> watermark value has been constant since the pipeline started.
> >> >
> >> > If no lateness is set, any late data should be admitted right?
> >> >
> >> > If no lateness is set, it means allowed lateness of Duration.ZERO,
> which
> >> > means that data that arrive after end-of-window will be dropped.
> >> >
> >> > Regarding 'droppedDueToLateness' metric, can't see it exposed
> anywhere, neither in Flink UI or Prometheus. I've seen it in Dataflow but
> seems to be a Dataflow specific metric right?
> >> >
> >> > Should not be Dataflow specific. But if you don't see it, it means it
> >> > could be zero. So, we can rule this out.
> >> >
> >> > We're using KinesisIO for reading messages.
> >> >
> >> > Kinesis uses UnboundedSource, which is expended to SDF starting from
> >> > Beam 2.25.0. The flag should change that as well. Can you try the
> >> > --experiments=use_deprecated_read and see if you Pipeline DAG changes
> >> > (should not contain Impulse transform at the beginning) and if it
> solves
> >> > your issues?
> >> >
> >> > On 2021/06/14 12:48:58, Jan Lukavský <je...@seznam.cz> wrote:
> >> >
> >> > Hi Eddy,
> >> >
> >> > does your data get buffered in a state - e.g. does the size of the
> state
> >> > grow over time? Do you see watermark being updated in your Flink
> WebUI?
> >> > When a stateful operation (and GroupByKey is a stateful operation)
> does
> >> > not output any data, the first place to look at is if watermark
> >> > correctly progresses. If it does not progress, then the input data
> must
> >> > be buffered in state and the size of the state should grow over time.
> If
> >> > it progresses, then it might be the case, that the data is too late
> >> > after the watermark (the watermark estimator might need tuning) and
> the
> >> > data gets dropped (note you don't set any allowed lateness, which
> >> > _might_ cause issues). You could see if your pipeline drops data in
> >> > "droppedDueToLateness" metric. The size of you state would not grow
> much
> >> > in that situation.
> >> >
> >> > Another hint - If you use KafkaIO, try to disable SDF wrapper for it
> >> > using "--experiments=use_deprecated_read" on command line (which you
> >> > then must pass to PipelineOptionsFactory). There is some suspicion
> that
> >> > SDF wrapper for Kafka might not work as expected in certain situations
> >> > with Flink.
> >> >
> >> > Please feel free to share any results,
> >> >
> >> >     Jan
> >> >
> >> > On 6/14/21 1:39 PM, Eddy G wrote:
> >> >
> >> > As seen in this image https://imgur.com/a/wrZET97, I'm trying to
> deal with late data (intentionally stopped my consumer so data has been
> accumulating for several days now). Now, with the following Window... I'm
> using Beam 2.27 and Flink 1.12.
> >> >
> >> >
>  Window.into(FixedWindows.of(Duration.standardMinutes(10)))
> >> >
> >> > And several parsing stages after, once it's time to write within the
> ParquetIO stage...
> >> >
> >> >                               FileIO
> >> >                                   .<String, MyClass>writeDynamic()
> >> >                                   .by(...)
> >> >                                   .via(...)
> >> >                                   .to(...)
> >> >                                   .withNaming(...)
> >> >
>  .withDestinationCoder(StringUtf8Coder.of())
> >> >
>  .withNumShards(options.getNumShards())
> >> >
> >> > it won't send bytes across all stages so no data is being written,
> still it accumulates in the first stage seen in the image and won't go
> further than that.
> >> >
> >> > Any reason why this may be happening? Wrong windowing strategy?
>

Re: GroupIntoShards not sending bytes further when dealing with huge amount of data

Posted by Robert Bradshaw <ro...@google.com>.
Is it possible to post the code? (Or the code of a similar, but
minimal, pipeline that exhibits the same issues?)

On Mon, Jun 14, 2021 at 2:15 PM Evan Galpin <ev...@gmail.com> wrote:
>
> @robert I have a pipeline which consistently shows a major slowdown (10 seconds Vs 10 minutes) between version <=2.23.0 and >=2.25.0 that can be boiled down to:
>
> - Read GCS file patterns from PubSub
> - Window into Fixed windows (repeating every 15 seconds)
> - Deduplicate/distinct (have tried both)
> - Read GCS blobs via patterns from the first step
> - Write file contents to sink
>
> It doesn't seem to matter if there are 0 messages in a subscription or 50k messages at startup. The rate of new messages however is very low. Not sure if those are helpful details, let me know if there's anything else specific which would help.
>
> On Mon, Jun 14, 2021 at 12:44 PM Robert Bradshaw <ro...@google.com> wrote:
>>
>> +1, we'd really like to get to the bottom of this, so clear
>> instructions on a pipeline/conditions that can reproduce it would be
>> great.
>>
>> On Mon, Jun 14, 2021 at 7:34 AM Jan Lukavský <je...@seznam.cz> wrote:
>> >
>> > Hi Eddy,
>> >
>> > you are probably hitting a not-yet discovered bug in SDF implementation in FlinkRunner that (under some currently unknown conditions) seems to stop advancing the watermark. This has been observed in one other instance (that I'm aware of). I think we don't yet have a tracking JIRA for that, would you mind filling it? It would be awesome if you could include estimations of messages per sec throughput that causes the issue in your case.
>> >
>> > +Tobias Kaymak
>> >
>> > Tobias, could you please confirm that the case you had with Flink stopping progressing watermark resembled this one?
>> >
>> > Thanks.
>> >
>> >  Jan
>> >
>> > On 6/14/21 4:11 PM, Eddy G wrote:
>> >
>> > Hi Jan,
>> >
>> > I've added --experiments=use_deprecated_read and it seems to work flawlessly (with my current Window and the one proposed by Evan).
>> >
>> > Why is this? Do Splittable DoFn now break current implementations? Are there any posts of possible breaking changes?
>> >
>> > On 2021/06/14 13:19:39, Jan Lukavský <je...@seznam.cz> wrote:
>> >
>> > Hi Eddy,
>> >
>> > answers inline.
>> >
>> > On 6/14/21 3:05 PM, Eddy G wrote:
>> >
>> > Hi Jan,
>> >
>> > Thanks for replying so fast!
>> >
>> > Regarding your questions,
>> >
>> > - "Does your data get buffered in a state?"
>> > Yes, I do have a state within a stage prior ParquetIO writing together with a Timer with PROCESSING_TIME.
>> >
>> > The stage which contains the state does send bytes to the next one which is the ParquetIO writing. Seems the @OnTimer doesn't get triggered and it's not clearing the state. This however does work under normal circumstances without having too much data queued waiting to be processed.
>> >
>> > OK, this suggests, that the watermark is for some reason "stuck". If you
>> > checkpoints enabled, you should see the size of the checkpoint to grow
>> > over time.
>> >
>> > - "Do you see watermark being updated in your Flink WebUI?"
>> > The stages that do have a watermark don't get updated. The same watermark value has been constant since the pipeline started.
>> >
>> > If no lateness is set, any late data should be admitted right?
>> >
>> > If no lateness is set, it means allowed lateness of Duration.ZERO, which
>> > means that data that arrive after end-of-window will be dropped.
>> >
>> > Regarding 'droppedDueToLateness' metric, can't see it exposed anywhere, neither in Flink UI or Prometheus. I've seen it in Dataflow but seems to be a Dataflow specific metric right?
>> >
>> > Should not be Dataflow specific. But if you don't see it, it means it
>> > could be zero. So, we can rule this out.
>> >
>> > We're using KinesisIO for reading messages.
>> >
>> > Kinesis uses UnboundedSource, which is expended to SDF starting from
>> > Beam 2.25.0. The flag should change that as well. Can you try the
>> > --experiments=use_deprecated_read and see if you Pipeline DAG changes
>> > (should not contain Impulse transform at the beginning) and if it solves
>> > your issues?
>> >
>> > On 2021/06/14 12:48:58, Jan Lukavský <je...@seznam.cz> wrote:
>> >
>> > Hi Eddy,
>> >
>> > does your data get buffered in a state - e.g. does the size of the state
>> > grow over time? Do you see watermark being updated in your Flink WebUI?
>> > When a stateful operation (and GroupByKey is a stateful operation) does
>> > not output any data, the first place to look at is if watermark
>> > correctly progresses. If it does not progress, then the input data must
>> > be buffered in state and the size of the state should grow over time. If
>> > it progresses, then it might be the case, that the data is too late
>> > after the watermark (the watermark estimator might need tuning) and the
>> > data gets dropped (note you don't set any allowed lateness, which
>> > _might_ cause issues). You could see if your pipeline drops data in
>> > "droppedDueToLateness" metric. The size of you state would not grow much
>> > in that situation.
>> >
>> > Another hint - If you use KafkaIO, try to disable SDF wrapper for it
>> > using "--experiments=use_deprecated_read" on command line (which you
>> > then must pass to PipelineOptionsFactory). There is some suspicion that
>> > SDF wrapper for Kafka might not work as expected in certain situations
>> > with Flink.
>> >
>> > Please feel free to share any results,
>> >
>> >     Jan
>> >
>> > On 6/14/21 1:39 PM, Eddy G wrote:
>> >
>> > As seen in this image https://imgur.com/a/wrZET97, I'm trying to deal with late data (intentionally stopped my consumer so data has been accumulating for several days now). Now, with the following Window... I'm using Beam 2.27 and Flink 1.12.
>> >
>> >                               Window.into(FixedWindows.of(Duration.standardMinutes(10)))
>> >
>> > And several parsing stages after, once it's time to write within the ParquetIO stage...
>> >
>> >                               FileIO
>> >                                   .<String, MyClass>writeDynamic()
>> >                                   .by(...)
>> >                                   .via(...)
>> >                                   .to(...)
>> >                                   .withNaming(...)
>> >                                   .withDestinationCoder(StringUtf8Coder.of())
>> >                                   .withNumShards(options.getNumShards())
>> >
>> > it won't send bytes across all stages so no data is being written, still it accumulates in the first stage seen in the image and won't go further than that.
>> >
>> > Any reason why this may be happening? Wrong windowing strategy?

Re: GroupIntoShards not sending bytes further when dealing with huge amount of data

Posted by Evan Galpin <ev...@gmail.com>.
@robert I have a pipeline which consistently shows a major slowdown (10
seconds Vs 10 minutes) between version <=2.23.0 and >=2.25.0 that can be
boiled down to:

- Read GCS file patterns from PubSub
- Window into Fixed windows (repeating every 15 seconds)
- Deduplicate/distinct (have tried both)
- Read GCS blobs via patterns from the first step
- Write file contents to sink

It doesn't seem to matter if there are 0 messages in a subscription or 50k
messages at startup. The rate of new messages however is very low. Not sure
if those are helpful details, let me know if there's anything else specific
which would help.

On Mon, Jun 14, 2021 at 12:44 PM Robert Bradshaw <ro...@google.com>
wrote:

> +1, we'd really like to get to the bottom of this, so clear
> instructions on a pipeline/conditions that can reproduce it would be
> great.
>
> On Mon, Jun 14, 2021 at 7:34 AM Jan Lukavský <je...@seznam.cz> wrote:
> >
> > Hi Eddy,
> >
> > you are probably hitting a not-yet discovered bug in SDF implementation
> in FlinkRunner that (under some currently unknown conditions) seems to stop
> advancing the watermark. This has been observed in one other instance (that
> I'm aware of). I think we don't yet have a tracking JIRA for that, would
> you mind filling it? It would be awesome if you could include estimations
> of messages per sec throughput that causes the issue in your case.
> >
> > +Tobias Kaymak
> >
> > Tobias, could you please confirm that the case you had with Flink
> stopping progressing watermark resembled this one?
> >
> > Thanks.
> >
> >  Jan
> >
> > On 6/14/21 4:11 PM, Eddy G wrote:
> >
> > Hi Jan,
> >
> > I've added --experiments=use_deprecated_read and it seems to work
> flawlessly (with my current Window and the one proposed by Evan).
> >
> > Why is this? Do Splittable DoFn now break current implementations? Are
> there any posts of possible breaking changes?
> >
> > On 2021/06/14 13:19:39, Jan Lukavský <je...@seznam.cz> wrote:
> >
> > Hi Eddy,
> >
> > answers inline.
> >
> > On 6/14/21 3:05 PM, Eddy G wrote:
> >
> > Hi Jan,
> >
> > Thanks for replying so fast!
> >
> > Regarding your questions,
> >
> > - "Does your data get buffered in a state?"
> > Yes, I do have a state within a stage prior ParquetIO writing together
> with a Timer with PROCESSING_TIME.
> >
> > The stage which contains the state does send bytes to the next one which
> is the ParquetIO writing. Seems the @OnTimer doesn't get triggered and it's
> not clearing the state. This however does work under normal circumstances
> without having too much data queued waiting to be processed.
> >
> > OK, this suggests, that the watermark is for some reason "stuck". If you
> > checkpoints enabled, you should see the size of the checkpoint to grow
> > over time.
> >
> > - "Do you see watermark being updated in your Flink WebUI?"
> > The stages that do have a watermark don't get updated. The same
> watermark value has been constant since the pipeline started.
> >
> > If no lateness is set, any late data should be admitted right?
> >
> > If no lateness is set, it means allowed lateness of Duration.ZERO, which
> > means that data that arrive after end-of-window will be dropped.
> >
> > Regarding 'droppedDueToLateness' metric, can't see it exposed anywhere,
> neither in Flink UI or Prometheus. I've seen it in Dataflow but seems to be
> a Dataflow specific metric right?
> >
> > Should not be Dataflow specific. But if you don't see it, it means it
> > could be zero. So, we can rule this out.
> >
> > We're using KinesisIO for reading messages.
> >
> > Kinesis uses UnboundedSource, which is expended to SDF starting from
> > Beam 2.25.0. The flag should change that as well. Can you try the
> > --experiments=use_deprecated_read and see if you Pipeline DAG changes
> > (should not contain Impulse transform at the beginning) and if it solves
> > your issues?
> >
> > On 2021/06/14 12:48:58, Jan Lukavský <je...@seznam.cz> wrote:
> >
> > Hi Eddy,
> >
> > does your data get buffered in a state - e.g. does the size of the state
> > grow over time? Do you see watermark being updated in your Flink WebUI?
> > When a stateful operation (and GroupByKey is a stateful operation) does
> > not output any data, the first place to look at is if watermark
> > correctly progresses. If it does not progress, then the input data must
> > be buffered in state and the size of the state should grow over time. If
> > it progresses, then it might be the case, that the data is too late
> > after the watermark (the watermark estimator might need tuning) and the
> > data gets dropped (note you don't set any allowed lateness, which
> > _might_ cause issues). You could see if your pipeline drops data in
> > "droppedDueToLateness" metric. The size of you state would not grow much
> > in that situation.
> >
> > Another hint - If you use KafkaIO, try to disable SDF wrapper for it
> > using "--experiments=use_deprecated_read" on command line (which you
> > then must pass to PipelineOptionsFactory). There is some suspicion that
> > SDF wrapper for Kafka might not work as expected in certain situations
> > with Flink.
> >
> > Please feel free to share any results,
> >
> >     Jan
> >
> > On 6/14/21 1:39 PM, Eddy G wrote:
> >
> > As seen in this image https://imgur.com/a/wrZET97, I'm trying to deal
> with late data (intentionally stopped my consumer so data has been
> accumulating for several days now). Now, with the following Window... I'm
> using Beam 2.27 and Flink 1.12.
> >
> >
>  Window.into(FixedWindows.of(Duration.standardMinutes(10)))
> >
> > And several parsing stages after, once it's time to write within the
> ParquetIO stage...
> >
> >                               FileIO
> >                                   .<String, MyClass>writeDynamic()
> >                                   .by(...)
> >                                   .via(...)
> >                                   .to(...)
> >                                   .withNaming(...)
> >
>  .withDestinationCoder(StringUtf8Coder.of())
> >                                   .withNumShards(options.getNumShards())
> >
> > it won't send bytes across all stages so no data is being written, still
> it accumulates in the first stage seen in the image and won't go further
> than that.
> >
> > Any reason why this may be happening? Wrong windowing strategy?
>

Re: GroupIntoShards not sending bytes further when dealing with huge amount of data

Posted by Robert Bradshaw <ro...@google.com>.
+1, we'd really like to get to the bottom of this, so clear
instructions on a pipeline/conditions that can reproduce it would be
great.

On Mon, Jun 14, 2021 at 7:34 AM Jan Lukavský <je...@seznam.cz> wrote:
>
> Hi Eddy,
>
> you are probably hitting a not-yet discovered bug in SDF implementation in FlinkRunner that (under some currently unknown conditions) seems to stop advancing the watermark. This has been observed in one other instance (that I'm aware of). I think we don't yet have a tracking JIRA for that, would you mind filling it? It would be awesome if you could include estimations of messages per sec throughput that causes the issue in your case.
>
> +Tobias Kaymak
>
> Tobias, could you please confirm that the case you had with Flink stopping progressing watermark resembled this one?
>
> Thanks.
>
>  Jan
>
> On 6/14/21 4:11 PM, Eddy G wrote:
>
> Hi Jan,
>
> I've added --experiments=use_deprecated_read and it seems to work flawlessly (with my current Window and the one proposed by Evan).
>
> Why is this? Do Splittable DoFn now break current implementations? Are there any posts of possible breaking changes?
>
> On 2021/06/14 13:19:39, Jan Lukavský <je...@seznam.cz> wrote:
>
> Hi Eddy,
>
> answers inline.
>
> On 6/14/21 3:05 PM, Eddy G wrote:
>
> Hi Jan,
>
> Thanks for replying so fast!
>
> Regarding your questions,
>
> - "Does your data get buffered in a state?"
> Yes, I do have a state within a stage prior ParquetIO writing together with a Timer with PROCESSING_TIME.
>
> The stage which contains the state does send bytes to the next one which is the ParquetIO writing. Seems the @OnTimer doesn't get triggered and it's not clearing the state. This however does work under normal circumstances without having too much data queued waiting to be processed.
>
> OK, this suggests, that the watermark is for some reason "stuck". If you
> checkpoints enabled, you should see the size of the checkpoint to grow
> over time.
>
> - "Do you see watermark being updated in your Flink WebUI?"
> The stages that do have a watermark don't get updated. The same watermark value has been constant since the pipeline started.
>
> If no lateness is set, any late data should be admitted right?
>
> If no lateness is set, it means allowed lateness of Duration.ZERO, which
> means that data that arrive after end-of-window will be dropped.
>
> Regarding 'droppedDueToLateness' metric, can't see it exposed anywhere, neither in Flink UI or Prometheus. I've seen it in Dataflow but seems to be a Dataflow specific metric right?
>
> Should not be Dataflow specific. But if you don't see it, it means it
> could be zero. So, we can rule this out.
>
> We're using KinesisIO for reading messages.
>
> Kinesis uses UnboundedSource, which is expended to SDF starting from
> Beam 2.25.0. The flag should change that as well. Can you try the
> --experiments=use_deprecated_read and see if you Pipeline DAG changes
> (should not contain Impulse transform at the beginning) and if it solves
> your issues?
>
> On 2021/06/14 12:48:58, Jan Lukavský <je...@seznam.cz> wrote:
>
> Hi Eddy,
>
> does your data get buffered in a state - e.g. does the size of the state
> grow over time? Do you see watermark being updated in your Flink WebUI?
> When a stateful operation (and GroupByKey is a stateful operation) does
> not output any data, the first place to look at is if watermark
> correctly progresses. If it does not progress, then the input data must
> be buffered in state and the size of the state should grow over time. If
> it progresses, then it might be the case, that the data is too late
> after the watermark (the watermark estimator might need tuning) and the
> data gets dropped (note you don't set any allowed lateness, which
> _might_ cause issues). You could see if your pipeline drops data in
> "droppedDueToLateness" metric. The size of you state would not grow much
> in that situation.
>
> Another hint - If you use KafkaIO, try to disable SDF wrapper for it
> using "--experiments=use_deprecated_read" on command line (which you
> then must pass to PipelineOptionsFactory). There is some suspicion that
> SDF wrapper for Kafka might not work as expected in certain situations
> with Flink.
>
> Please feel free to share any results,
>
>     Jan
>
> On 6/14/21 1:39 PM, Eddy G wrote:
>
> As seen in this image https://imgur.com/a/wrZET97, I'm trying to deal with late data (intentionally stopped my consumer so data has been accumulating for several days now). Now, with the following Window... I'm using Beam 2.27 and Flink 1.12.
>
>                               Window.into(FixedWindows.of(Duration.standardMinutes(10)))
>
> And several parsing stages after, once it's time to write within the ParquetIO stage...
>
>                               FileIO
>                                   .<String, MyClass>writeDynamic()
>                                   .by(...)
>                                   .via(...)
>                                   .to(...)
>                                   .withNaming(...)
>                                   .withDestinationCoder(StringUtf8Coder.of())
>                                   .withNumShards(options.getNumShards())
>
> it won't send bytes across all stages so no data is being written, still it accumulates in the first stage seen in the image and won't go further than that.
>
> Any reason why this may be happening? Wrong windowing strategy?

Re: GroupIntoShards not sending bytes further when dealing with huge amount of data

Posted by Jan Lukavský <je...@seznam.cz>.
Hi Eddy,

you are probably hitting a not-yet discovered bug in SDF implementation 
in FlinkRunner that (under some currently unknown conditions) seems to 
stop advancing the watermark. This has been observed in one other 
instance (that I'm aware of). I think we don't yet have a tracking JIRA 
for that, would you mind filling it? It would be awesome if you could 
include estimations of messages per sec throughput that causes the issue 
in your case.

+Tobias Kaymak <ma...@ricardo.ch>

Tobias, could you please confirm that the case you had with Flink 
stopping progressing watermark resembled this one?

Thanks.

  Jan

On 6/14/21 4:11 PM, Eddy G wrote:
> Hi Jan,
>
> I've added --experiments=use_deprecated_read and it seems to work flawlessly (with my current Window and the one proposed by Evan).
>
> Why is this? Do Splittable DoFn now break current implementations? Are there any posts of possible breaking changes?
>
> On 2021/06/14 13:19:39, Jan Lukavský <je...@seznam.cz> wrote:
>> Hi Eddy,
>>
>> answers inline.
>>
>> On 6/14/21 3:05 PM, Eddy G wrote:
>>> Hi Jan,
>>>
>>> Thanks for replying so fast!
>>>
>>> Regarding your questions,
>>>
>>> - "Does your data get buffered in a state?"
>>> Yes, I do have a state within a stage prior ParquetIO writing together with a Timer with PROCESSING_TIME.
>>>
>>> The stage which contains the state does send bytes to the next one which is the ParquetIO writing. Seems the @OnTimer doesn't get triggered and it's not clearing the state. This however does work under normal circumstances without having too much data queued waiting to be processed.
>> OK, this suggests, that the watermark is for some reason "stuck". If you
>> checkpoints enabled, you should see the size of the checkpoint to grow
>> over time.
>>> - "Do you see watermark being updated in your Flink WebUI?"
>>> The stages that do have a watermark don't get updated. The same watermark value has been constant since the pipeline started.
>>>
>>> If no lateness is set, any late data should be admitted right?
>> If no lateness is set, it means allowed lateness of Duration.ZERO, which
>> means that data that arrive after end-of-window will be dropped.
>>> Regarding 'droppedDueToLateness' metric, can't see it exposed anywhere, neither in Flink UI or Prometheus. I've seen it in Dataflow but seems to be a Dataflow specific metric right?
>> Should not be Dataflow specific. But if you don't see it, it means it
>> could be zero. So, we can rule this out.
>>> We're using KinesisIO for reading messages.
>> Kinesis uses UnboundedSource, which is expended to SDF starting from
>> Beam 2.25.0. The flag should change that as well. Can you try the
>> --experiments=use_deprecated_read and see if you Pipeline DAG changes
>> (should not contain Impulse transform at the beginning) and if it solves
>> your issues?
>>> On 2021/06/14 12:48:58, Jan Lukavský <je...@seznam.cz> wrote:
>>>> Hi Eddy,
>>>>
>>>> does your data get buffered in a state - e.g. does the size of the state
>>>> grow over time? Do you see watermark being updated in your Flink WebUI?
>>>> When a stateful operation (and GroupByKey is a stateful operation) does
>>>> not output any data, the first place to look at is if watermark
>>>> correctly progresses. If it does not progress, then the input data must
>>>> be buffered in state and the size of the state should grow over time. If
>>>> it progresses, then it might be the case, that the data is too late
>>>> after the watermark (the watermark estimator might need tuning) and the
>>>> data gets dropped (note you don't set any allowed lateness, which
>>>> _might_ cause issues). You could see if your pipeline drops data in
>>>> "droppedDueToLateness" metric. The size of you state would not grow much
>>>> in that situation.
>>>>
>>>> Another hint - If you use KafkaIO, try to disable SDF wrapper for it
>>>> using "--experiments=use_deprecated_read" on command line (which you
>>>> then must pass to PipelineOptionsFactory). There is some suspicion that
>>>> SDF wrapper for Kafka might not work as expected in certain situations
>>>> with Flink.
>>>>
>>>> Please feel free to share any results,
>>>>
>>>>      Jan
>>>>
>>>> On 6/14/21 1:39 PM, Eddy G wrote:
>>>>> As seen in this image https://imgur.com/a/wrZET97, I'm trying to deal with late data (intentionally stopped my consumer so data has been accumulating for several days now). Now, with the following Window... I'm using Beam 2.27 and Flink 1.12.
>>>>>
>>>>>                                Window.into(FixedWindows.of(Duration.standardMinutes(10)))
>>>>>
>>>>> And several parsing stages after, once it's time to write within the ParquetIO stage...
>>>>>
>>>>>                                FileIO
>>>>>                                    .<String, MyClass>writeDynamic()
>>>>>                                    .by(...)
>>>>>                                    .via(...)
>>>>>                                    .to(...)
>>>>>                                    .withNaming(...)
>>>>>                                    .withDestinationCoder(StringUtf8Coder.of())
>>>>>                                    .withNumShards(options.getNumShards())
>>>>>
>>>>> it won't send bytes across all stages so no data is being written, still it accumulates in the first stage seen in the image and won't go further than that.
>>>>>
>>>>> Any reason why this may be happening? Wrong windowing strategy?

Re: GroupIntoShards not sending bytes further when dealing with huge amount of data

Posted by Eddy G <ka...@gmail.com>.
Hi Jan,

I've added --experiments=use_deprecated_read and it seems to work flawlessly (with my current Window and the one proposed by Evan).

Why is this? Do Splittable DoFn now break current implementations? Are there any posts of possible breaking changes?

On 2021/06/14 13:19:39, Jan Lukavský <je...@seznam.cz> wrote: 
> Hi Eddy,
> 
> answers inline.
> 
> On 6/14/21 3:05 PM, Eddy G wrote:
> > Hi Jan,
> >
> > Thanks for replying so fast!
> >
> > Regarding your questions,
> >
> > - "Does your data get buffered in a state?"
> > Yes, I do have a state within a stage prior ParquetIO writing together with a Timer with PROCESSING_TIME.
> >
> > The stage which contains the state does send bytes to the next one which is the ParquetIO writing. Seems the @OnTimer doesn't get triggered and it's not clearing the state. This however does work under normal circumstances without having too much data queued waiting to be processed.
> OK, this suggests, that the watermark is for some reason "stuck". If you 
> checkpoints enabled, you should see the size of the checkpoint to grow 
> over time.
> >
> > - "Do you see watermark being updated in your Flink WebUI?"
> > The stages that do have a watermark don't get updated. The same watermark value has been constant since the pipeline started.
> >
> > If no lateness is set, any late data should be admitted right?
> If no lateness is set, it means allowed lateness of Duration.ZERO, which 
> means that data that arrive after end-of-window will be dropped.
> >
> > Regarding 'droppedDueToLateness' metric, can't see it exposed anywhere, neither in Flink UI or Prometheus. I've seen it in Dataflow but seems to be a Dataflow specific metric right?
> Should not be Dataflow specific. But if you don't see it, it means it 
> could be zero. So, we can rule this out.
> >
> > We're using KinesisIO for reading messages.
> Kinesis uses UnboundedSource, which is expended to SDF starting from 
> Beam 2.25.0. The flag should change that as well. Can you try the 
> --experiments=use_deprecated_read and see if you Pipeline DAG changes 
> (should not contain Impulse transform at the beginning) and if it solves 
> your issues?
> >
> > On 2021/06/14 12:48:58, Jan Lukavský <je...@seznam.cz> wrote:
> >> Hi Eddy,
> >>
> >> does your data get buffered in a state - e.g. does the size of the state
> >> grow over time? Do you see watermark being updated in your Flink WebUI?
> >> When a stateful operation (and GroupByKey is a stateful operation) does
> >> not output any data, the first place to look at is if watermark
> >> correctly progresses. If it does not progress, then the input data must
> >> be buffered in state and the size of the state should grow over time. If
> >> it progresses, then it might be the case, that the data is too late
> >> after the watermark (the watermark estimator might need tuning) and the
> >> data gets dropped (note you don't set any allowed lateness, which
> >> _might_ cause issues). You could see if your pipeline drops data in
> >> "droppedDueToLateness" metric. The size of you state would not grow much
> >> in that situation.
> >>
> >> Another hint - If you use KafkaIO, try to disable SDF wrapper for it
> >> using "--experiments=use_deprecated_read" on command line (which you
> >> then must pass to PipelineOptionsFactory). There is some suspicion that
> >> SDF wrapper for Kafka might not work as expected in certain situations
> >> with Flink.
> >>
> >> Please feel free to share any results,
> >>
> >>     Jan
> >>
> >> On 6/14/21 1:39 PM, Eddy G wrote:
> >>> As seen in this image https://imgur.com/a/wrZET97, I'm trying to deal with late data (intentionally stopped my consumer so data has been accumulating for several days now). Now, with the following Window... I'm using Beam 2.27 and Flink 1.12.
> >>>
> >>>                               Window.into(FixedWindows.of(Duration.standardMinutes(10)))
> >>>
> >>> And several parsing stages after, once it's time to write within the ParquetIO stage...
> >>>
> >>>                               FileIO
> >>>                                   .<String, MyClass>writeDynamic()
> >>>                                   .by(...)
> >>>                                   .via(...)
> >>>                                   .to(...)
> >>>                                   .withNaming(...)
> >>>                                   .withDestinationCoder(StringUtf8Coder.of())
> >>>                                   .withNumShards(options.getNumShards())
> >>>
> >>> it won't send bytes across all stages so no data is being written, still it accumulates in the first stage seen in the image and won't go further than that.
> >>>
> >>> Any reason why this may be happening? Wrong windowing strategy?
> 

Re: GroupIntoShards not sending bytes further when dealing with huge amount of data

Posted by Jan Lukavský <je...@seznam.cz>.
Hi Eddy,

answers inline.

On 6/14/21 3:05 PM, Eddy G wrote:
> Hi Jan,
>
> Thanks for replying so fast!
>
> Regarding your questions,
>
> - "Does your data get buffered in a state?"
> Yes, I do have a state within a stage prior ParquetIO writing together with a Timer with PROCESSING_TIME.
>
> The stage which contains the state does send bytes to the next one which is the ParquetIO writing. Seems the @OnTimer doesn't get triggered and it's not clearing the state. This however does work under normal circumstances without having too much data queued waiting to be processed.
OK, this suggests, that the watermark is for some reason "stuck". If you 
checkpoints enabled, you should see the size of the checkpoint to grow 
over time.
>
> - "Do you see watermark being updated in your Flink WebUI?"
> The stages that do have a watermark don't get updated. The same watermark value has been constant since the pipeline started.
>
> If no lateness is set, any late data should be admitted right?
If no lateness is set, it means allowed lateness of Duration.ZERO, which 
means that data that arrive after end-of-window will be dropped.
>
> Regarding 'droppedDueToLateness' metric, can't see it exposed anywhere, neither in Flink UI or Prometheus. I've seen it in Dataflow but seems to be a Dataflow specific metric right?
Should not be Dataflow specific. But if you don't see it, it means it 
could be zero. So, we can rule this out.
>
> We're using KinesisIO for reading messages.
Kinesis uses UnboundedSource, which is expended to SDF starting from 
Beam 2.25.0. The flag should change that as well. Can you try the 
--experiments=use_deprecated_read and see if you Pipeline DAG changes 
(should not contain Impulse transform at the beginning) and if it solves 
your issues?
>
> On 2021/06/14 12:48:58, Jan Lukavský <je...@seznam.cz> wrote:
>> Hi Eddy,
>>
>> does your data get buffered in a state - e.g. does the size of the state
>> grow over time? Do you see watermark being updated in your Flink WebUI?
>> When a stateful operation (and GroupByKey is a stateful operation) does
>> not output any data, the first place to look at is if watermark
>> correctly progresses. If it does not progress, then the input data must
>> be buffered in state and the size of the state should grow over time. If
>> it progresses, then it might be the case, that the data is too late
>> after the watermark (the watermark estimator might need tuning) and the
>> data gets dropped (note you don't set any allowed lateness, which
>> _might_ cause issues). You could see if your pipeline drops data in
>> "droppedDueToLateness" metric. The size of you state would not grow much
>> in that situation.
>>
>> Another hint - If you use KafkaIO, try to disable SDF wrapper for it
>> using "--experiments=use_deprecated_read" on command line (which you
>> then must pass to PipelineOptionsFactory). There is some suspicion that
>> SDF wrapper for Kafka might not work as expected in certain situations
>> with Flink.
>>
>> Please feel free to share any results,
>>
>>     Jan
>>
>> On 6/14/21 1:39 PM, Eddy G wrote:
>>> As seen in this image https://imgur.com/a/wrZET97, I'm trying to deal with late data (intentionally stopped my consumer so data has been accumulating for several days now). Now, with the following Window... I'm using Beam 2.27 and Flink 1.12.
>>>
>>>                               Window.into(FixedWindows.of(Duration.standardMinutes(10)))
>>>
>>> And several parsing stages after, once it's time to write within the ParquetIO stage...
>>>
>>>                               FileIO
>>>                                   .<String, MyClass>writeDynamic()
>>>                                   .by(...)
>>>                                   .via(...)
>>>                                   .to(...)
>>>                                   .withNaming(...)
>>>                                   .withDestinationCoder(StringUtf8Coder.of())
>>>                                   .withNumShards(options.getNumShards())
>>>
>>> it won't send bytes across all stages so no data is being written, still it accumulates in the first stage seen in the image and won't go further than that.
>>>
>>> Any reason why this may be happening? Wrong windowing strategy?

Re: GroupIntoShards not sending bytes further when dealing with huge amount of data

Posted by Eddy G <ka...@gmail.com>.
Hi Jan,

Thanks for replying so fast!

Regarding your questions,

- "Does your data get buffered in a state?"
Yes, I do have a state within a stage prior ParquetIO writing together with a Timer with PROCESSING_TIME.

The stage which contains the state does send bytes to the next one which is the ParquetIO writing. Seems the @OnTimer doesn't get triggered and it's not clearing the state. This however does work under normal circumstances without having too much data queued waiting to be processed.

- "Do you see watermark being updated in your Flink WebUI?"
The stages that do have a watermark don't get updated. The same watermark value has been constant since the pipeline started.

If no lateness is set, any late data should be admitted right?

Regarding 'droppedDueToLateness' metric, can't see it exposed anywhere, neither in Flink UI or Prometheus. I've seen it in Dataflow but seems to be a Dataflow specific metric right?

We're using KinesisIO for reading messages.

On 2021/06/14 12:48:58, Jan Lukavský <je...@seznam.cz> wrote: 
> Hi Eddy,
> 
> does your data get buffered in a state - e.g. does the size of the state 
> grow over time? Do you see watermark being updated in your Flink WebUI? 
> When a stateful operation (and GroupByKey is a stateful operation) does 
> not output any data, the first place to look at is if watermark 
> correctly progresses. If it does not progress, then the input data must 
> be buffered in state and the size of the state should grow over time. If 
> it progresses, then it might be the case, that the data is too late 
> after the watermark (the watermark estimator might need tuning) and the 
> data gets dropped (note you don't set any allowed lateness, which 
> _might_ cause issues). You could see if your pipeline drops data in 
> "droppedDueToLateness" metric. The size of you state would not grow much 
> in that situation.
> 
> Another hint - If you use KafkaIO, try to disable SDF wrapper for it 
> using "--experiments=use_deprecated_read" on command line (which you 
> then must pass to PipelineOptionsFactory). There is some suspicion that 
> SDF wrapper for Kafka might not work as expected in certain situations 
> with Flink.
> 
> Please feel free to share any results,
> 
>    Jan
> 
> On 6/14/21 1:39 PM, Eddy G wrote:
> > As seen in this image https://imgur.com/a/wrZET97, I'm trying to deal with late data (intentionally stopped my consumer so data has been accumulating for several days now). Now, with the following Window... I'm using Beam 2.27 and Flink 1.12.
> >
> >                              Window.into(FixedWindows.of(Duration.standardMinutes(10)))
> >
> > And several parsing stages after, once it's time to write within the ParquetIO stage...
> >
> >                              FileIO
> >                                  .<String, MyClass>writeDynamic()
> >                                  .by(...)
> >                                  .via(...)
> >                                  .to(...)
> >                                  .withNaming(...)
> >                                  .withDestinationCoder(StringUtf8Coder.of())
> >                                  .withNumShards(options.getNumShards())
> >
> > it won't send bytes across all stages so no data is being written, still it accumulates in the first stage seen in the image and won't go further than that.
> >
> > Any reason why this may be happening? Wrong windowing strategy?
> 

Re: GroupIntoShards not sending bytes further when dealing with huge amount of data

Posted by Jan Lukavský <je...@seznam.cz>.
Hi Eddy,

does your data get buffered in a state - e.g. does the size of the state 
grow over time? Do you see watermark being updated in your Flink WebUI? 
When a stateful operation (and GroupByKey is a stateful operation) does 
not output any data, the first place to look at is if watermark 
correctly progresses. If it does not progress, then the input data must 
be buffered in state and the size of the state should grow over time. If 
it progresses, then it might be the case, that the data is too late 
after the watermark (the watermark estimator might need tuning) and the 
data gets dropped (note you don't set any allowed lateness, which 
_might_ cause issues). You could see if your pipeline drops data in 
"droppedDueToLateness" metric. The size of you state would not grow much 
in that situation.

Another hint - If you use KafkaIO, try to disable SDF wrapper for it 
using "--experiments=use_deprecated_read" on command line (which you 
then must pass to PipelineOptionsFactory). There is some suspicion that 
SDF wrapper for Kafka might not work as expected in certain situations 
with Flink.

Please feel free to share any results,

   Jan

On 6/14/21 1:39 PM, Eddy G wrote:
> As seen in this image https://imgur.com/a/wrZET97, I'm trying to deal with late data (intentionally stopped my consumer so data has been accumulating for several days now). Now, with the following Window... I'm using Beam 2.27 and Flink 1.12.
>
>                              Window.into(FixedWindows.of(Duration.standardMinutes(10)))
>
> And several parsing stages after, once it's time to write within the ParquetIO stage...
>
>                              FileIO
>                                  .<String, MyClass>writeDynamic()
>                                  .by(...)
>                                  .via(...)
>                                  .to(...)
>                                  .withNaming(...)
>                                  .withDestinationCoder(StringUtf8Coder.of())
>                                  .withNumShards(options.getNumShards())
>
> it won't send bytes across all stages so no data is being written, still it accumulates in the first stage seen in the image and won't go further than that.
>
> Any reason why this may be happening? Wrong windowing strategy?