[
https://issues.apache.org/jira/browse/BEAM-8347?focusedWorklogId=337201&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-337201
]
ASF GitHub Bot logged work on BEAM-8347:
----------------------------------------
Author: ASF GitHub Bot
Created on: 01/Nov/19 06:00
Start Date: 01/Nov/19 06:00
Worklog Time Spent: 10m
Work Description: jbonofre commented on pull request #9820: [BEAM-8347]:
Consistently advance UnboundedRabbitMqReader watermark
URL: https://github.com/apache/beam/pull/9820#discussion_r341459765
##########
File path:
sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqIO.java
##########
@@ -376,16 +389,11 @@ public void finalizeCheckpoint() throws IOException {
this.source = source;
this.current = null;
this.checkpointMark = checkpointMark != null ? checkpointMark : new
RabbitMQCheckpointMark();
- try {
- connectionHandler = new ConnectionHandler(source.spec.uri());
- } catch (Exception e) {
- throw new IOException(e);
- }
}
@Override
public Instant getWatermark() {
- return checkpointMark.oldestTimestamp;
+ return checkpointMark.latestTimestamp;
Review comment:
I agree, that's the challenge dealing with watermark in IO like RabbitMQ,
JMS, MQTT, etc.
I implemented the watermark the same way in those IOs, because it's always
the same: we can only use the timestamp of the message.
I think this change makes sense in two cases:
1. when there's not a lot of messages (but in that case, we can imagine to
update the watermark before the finalize checkpoint
2. when we use an existing exchange (instead of creating one)
As it depends of the use case, I think it would be better to have a way to
provide a watermark advance function or at least allow user to configure way
(in the IO) to update the watermark.
----------------------------------------------------------------
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: 337201)
Time Spent: 50m (was: 40m)
> 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
> Priority: Major
> Time Spent: 50m
> 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)