[
https://issues.apache.org/jira/browse/BEAM-12690?focusedWorklogId=635346&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-635346
]
ASF GitHub Bot logged work on BEAM-12690:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 06/Aug/21 15:51
Start Date: 06/Aug/21 15:51
Worklog Time Spent: 10m
Work Description: reuvenlax commented on a change in pull request #15249:
URL: https://github.com/apache/beam/pull/15249#discussion_r684338356
##########
File path:
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupIntoBatches.java
##########
@@ -398,11 +424,31 @@ public void processElement(
storedBatchSizeBytes.readLater();
}
- long num = storedBatchSize.read();
- if (maxBufferingDuration.isLongerThan(Duration.ZERO) && num == 1) {
- // This is the first element in batch. Start counting buffering time
if a limit was set.
- bufferingTimer.offset(maxBufferingDuration).setRelative();
+ long num;
+ if (maxBufferingDuration.isLongerThan(Duration.ZERO)) {
+ minBufferedTs.readLater();
+ num = storedBatchSize.read();
+
+ long oldOutputTs =
+ MoreObjects.firstNonNull(
+ minBufferedTs.read(),
BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis());
+ minBufferedTs.add(elementTs.getMillis());
+ // If this is the first element in the batch or if the timer's output
timestamp needs
+ // modifying, then set a
+ // timer.
+ if (num == 1 || minBufferedTs.read() != oldOutputTs) {
+ long targetTs =
+ MoreObjects.firstNonNull(
+ timerTs.read(),
+ bufferingTimer.getCurrentRelativeTime().getMillis()
Review comment:
That is what's happening - getCurrentRelativeTime is basically current
time. We don't want to query the OS for current time because then unit tests
won't work. Unit tests (that use TestStream) inject a fake clock so that they
can control the advancement of time, and that won't work if we query the OS
itself.
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 635346)
Time Spent: 1.5h (was: 1h 20m)
> GroupIntoBatches does not properly maintain watermarks
> ------------------------------------------------------
>
> Key: BEAM-12690
> URL: https://issues.apache.org/jira/browse/BEAM-12690
> Project: Beam
> Issue Type: Improvement
> Components: sdk-java-core
> Reporter: Reuven Lax
> Priority: P2
> Time Spent: 1.5h
> Remaining Estimate: 0h
>
> Since GIB buffers elements, the watermark should be held up to the minimum
> event-time buffered. Timer.withOutputTimestamp is the correct way to do this
> in Beam.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)