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

[FLINK-36664][Window]Window with offset need deal offset when cal nextTriggerWatermark. #25629

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
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 @@ -123,7 +123,11 @@ public void processWatermark(Watermark mark) throws Exception {
windowBuffer.advanceProgress(currentWatermark);
nextTriggerWatermark =
getNextTriggerWatermark(
currentWatermark, windowInterval, shiftTimezone, useDayLightSaving);
currentWatermark,
windowInterval,
sliceAssigner.getWindowOffset(),
shiftTimezone,
useDayLightSaving);
}
}
super.processWatermark(mark);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -146,7 +146,11 @@ public void advanceProgress(long progress) throws Exception {
windowBuffer.advanceProgress(currentProgress);
nextTriggerProgress =
getNextTriggerWatermark(
currentProgress, windowInterval, shiftTimeZone, useDayLightSaving);
currentProgress,
windowInterval,
sliceAssigner.getWindowOffset(),
shiftTimeZone,
useDayLightSaving);
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,4 +65,7 @@ public interface SliceAssigner extends WindowAssigner {
* slice assigned.
*/
long getSliceEndInterval();

/** Returns the offset of window. */
long getWindowOffset();
}
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,12 @@ public static TumblingSliceAssigner tumbling(
return new TumblingSliceAssigner(rowtimeIndex, shiftTimeZone, size.toMillis(), 0);
}

public static TumblingSliceAssigner tumbling(
int rowtimeIndex, ZoneId shiftTimeZone, Duration size, Duration offset) {
return new TumblingSliceAssigner(
rowtimeIndex, shiftTimeZone, size.toMillis(), offset.toMillis());
}

/**
* Creates a hopping window {@link SliceAssigner} that assigns elements to slices of hopping
* windows.
Expand Down Expand Up @@ -189,6 +195,11 @@ public long getSliceEndInterval() {
return size;
}

@Override
public long getWindowOffset() {
return offset;
}

@Override
public String getDescription() {
return String.format("TumblingWindow(size=%dms, offset=%dms)", size, offset);
Expand Down Expand Up @@ -265,6 +276,11 @@ public long getSliceEndInterval() {
return sliceSize;
}

@Override
public long getWindowOffset() {
return offset;
}

@Override
public void mergeSlices(long sliceEnd, MergeCallback<Long, Iterable<Long>> callback)
throws Exception {
Expand Down Expand Up @@ -370,6 +386,11 @@ public long getSliceEndInterval() {
return step;
}

@Override
public long getWindowOffset() {
return offset;
}

@Override
public void mergeSlices(long sliceEnd, MergeCallback<Long, Iterable<Long>> callback)
throws Exception {
Expand Down Expand Up @@ -451,6 +472,11 @@ public long getSliceEndInterval() {
return innerAssigner.getSliceEndInterval();
}

@Override
public long getWindowOffset() {
return innerAssigner.getWindowOffset();
}

@Override
public boolean isEventTime() {
// it always works in event-time mode if input row has been attached windows
Expand Down Expand Up @@ -566,6 +592,11 @@ public long getSliceEndInterval() {
return innerAssigner.getSliceEndInterval();
}

@Override
public long getWindowOffset() {
return 0;
}

@Override
public boolean isEventTime() {
// it always works in event-time mode if input row has been attached slices
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -184,7 +184,11 @@ public static boolean isWindowFired(

/** Method to get the next watermark to trigger window. */
public static long getNextTriggerWatermark(
long currentWatermark, long interval, ZoneId shiftTimezone, boolean useDayLightSaving) {
long currentWatermark,
long interval,
long windowOffset,
ZoneId shiftTimezone,
boolean useDayLightSaving) {
if (currentWatermark == Long.MAX_VALUE) {
return currentWatermark;
}
Expand All @@ -194,10 +198,12 @@ public static long getNextTriggerWatermark(
if (useDayLightSaving) {
long utcWindowStart =
getWindowStartWithOffset(
toUtcTimestampMills(currentWatermark, shiftTimezone), 0L, interval);
toUtcTimestampMills(currentWatermark, shiftTimezone),
windowOffset,
interval);
triggerWatermark = toEpochMillsForTimer(utcWindowStart + interval - 1, shiftTimezone);
} else {
long start = getWindowStartWithOffset(currentWatermark, 0L, interval);
long start = getWindowStartWithOffset(currentWatermark, windowOffset, interval);
triggerWatermark = start + interval - 1;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -641,6 +641,51 @@ void testEventTimeTumblingWindows() throws Exception {
testHarness.close();
}

@TestTemplate
public void testEventTimeTumblingWindowsWithOffset() throws Exception {
final SliceAssigner assigner =
SliceAssigners.tumbling(
2, shiftTimeZone, Duration.ofSeconds(3), Duration.ofSeconds(1));
final SlicingSumAndCountAggsFunction aggsFunction =
new SlicingSumAndCountAggsFunction(assigner);
WindowAggOperator<RowData, ?> operator =
WindowAggOperatorBuilder.builder()
.inputSerializer(INPUT_ROW_SER)
.shiftTimeZone(shiftTimeZone)
.keySerializer(KEY_SER)
.assigner(assigner)
.aggregate(createGeneratedAggsHandle(aggsFunction), ACC_SER)
.build();

OneInputStreamOperatorTestHarness<RowData, RowData> testHarness =
createTestHarness(operator);

testHarness.setup(OUT_SERIALIZER);
testHarness.open();

// process elements
ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();

testHarness.processElement(insertRecord("key1", 1, fromEpochMillis(2999L)));
testHarness.processWatermark(new Watermark(3000));
expectedOutput.add(new Watermark(3000));
ASSERTER.assertOutputEqualsSorted(
"Output was not correct.", expectedOutput, testHarness.getOutput());

testHarness.processWatermark(new Watermark(4000));
expectedOutput.add(insertRecord("key1", 1L, 1L, localMills(1000L), localMills(4000L)));
expectedOutput.add(new Watermark(4000));
ASSERTER.assertOutputEqualsSorted(
"Output was not correct.", expectedOutput, testHarness.getOutput());

testHarness.processWatermark(new Watermark(20000));
expectedOutput.add(new Watermark(20000));
ASSERTER.assertOutputEqualsSorted(
"Output was not correct.", expectedOutput, testHarness.getOutput());

testHarness.close();
}

@TestTemplate
void testProcessingTimeTumblingWindows() throws Exception {

Expand Down