[
https://issues.apache.org/jira/browse/BEAM-8347?focusedWorklogId=337380&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-337380
]
ASF GitHub Bot logged work on BEAM-8347:
----------------------------------------
Author: ASF GitHub Bot
Created on: 01/Nov/19 15:43
Start Date: 01/Nov/19 15:43
Worklog Time Spent: 10m
Work Description: jbonofre commented on issue #9820: [BEAM-8347]:
Consistently advance UnboundedRabbitMqReader watermark
URL: https://github.com/apache/beam/pull/9820#issuecomment-548837125
@drobert yeah, I agree. And actually, I also had hard times to think about
an efficient watermark strategy in these IOs (jms, amqp, rabbitmq, mqtt, ...).
I used the same approach in these IOs: the watermark is the oldest timestamp
before finalizing the checkpoint. For instance, in JmsIO checkpoint, when the
checkpoint is finalized I do:
```
message.acknowledge();
Instant currentMessageTimestamp = new
Instant(message.getJMSTimestamp());
snapshot.updateOldestPendingTimestampIf(currentMessageTimestamp,
Instant::isAfter);
```
So, the watermark advances with the oldest pending message timestamp.
The finalize checkpoint happens when the bundle is completed, so, I think
it's consistent.
Right ?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 337380)
Time Spent: 2h (was: 1h 50m)
> UnboundedRabbitMqReader can fail to advance watermark if no new data comes in
> -----------------------------------------------------------------------------
>
> Key: BEAM-8347
> URL: https://issues.apache.org/jira/browse/BEAM-8347
> Project: Beam
> Issue Type: Bug
> Components: io-java-rabbitmq
> Affects Versions: 2.15.0
> Environment: testing has been done using the DirectRunner. I also
> have DataflowRunner available
> Reporter: Daniel Robert
> Assignee: Jean-Baptiste Onofré
> Priority: Major
> Time Spent: 2h
> Remaining Estimate: 0h
>
> I stumbled upon this and then saw a similar StackOverflow post:
> [https://stackoverflow.com/questions/55736593/apache-beam-rabbitmqio-watermark-doesnt-advance]
> When calling `advance()` if there are no messages, no state changes,
> including no changes to the CheckpointMark or Watermark. If there is a
> relatively constant rate of new messages coming in, this is not a problem. If
> data is bursty, and there are periods of no new messages coming in, the
> watermark will never advance.
> Contrast this with some of the logic in PubsubIO which will make provisions
> for periods of inactivity to advance the watermark (although it, too, is
> imperfect: https://issues.apache.org/jira/browse/BEAM-7322 )
> The example given in the StackOverflow post is something like this:
>
> {code:java}
> pipeline
> .apply(RabbitMqIO.read()
> .withUri("amqp://guest:guest@localhost:5672")
> .withQueue("test")
> .apply("Windowing",
> Window.<RabbitMqMessage>into(
> FixedWindows.of(Duration.standardSeconds(10)))
> .triggering(AfterWatermark.pastEndOfWindow())
> .withAllowedLateness(Duration.ZERO)
> .accumulatingFiredPanes()){code}
> If I push 2 messages into my rabbit queue, I see 2 unack'd messages and a
> window that never performs an on time trigger.
>
--
This message was sent by Atlassian Jira
(v8.3.4#803005)