Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Stop emitting upon truncate in Java PeriodicSequence #25716

Merged
merged 1 commit into from
Mar 4, 2023
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -194,6 +194,12 @@ public WatermarkEstimator<Instant> newWatermarkEstimator(
return new WatermarkEstimators.Manual(state);
}

@TruncateRestriction
public RestrictionTracker.TruncateResult<OffsetRange> truncate() {
// stop emitting immediately upon drain
return null;
}

@ProcessElement
public ProcessContinuation processElement(
@Element SequenceDefinition srcElement,
Expand All @@ -207,7 +213,7 @@ public ProcessContinuation processElement(

boolean claimSuccess = true;

estimator.setWatermark(Instant.ofEpochMilli(restriction.getFrom()));
estimator.setWatermark(Instant.ofEpochMilli(nextOutput));
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This change is made in accordance with Python implementation. Currently the watermark will be set back to the initial timestamp (which violates the monotonically increasing) though it will be set back in the while clause.


while (claimSuccess && Instant.ofEpochMilli(nextOutput).isBeforeNow()) {
claimSuccess = restrictionTracker.tryClaim(nextOutput);
Expand Down