-
Notifications
You must be signed in to change notification settings - Fork 13k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINKCC-1283] Properly initialize initial splits in WatermarkOutputM…
…ultiplexer Without this fix, initial splits were registered in the multiplexer only when first record from that split has been emitted. This was leading to incorrectly emitted watermarks, as resulting watermark was not properly combined from the initial splits, but only from the splits that have already emitted at least one record.
- Loading branch information
Showing
3 changed files
with
97 additions
and
1 deletion.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
95 changes: 95 additions & 0 deletions
95
.../src/test/java/org/apache/flink/streaming/api/operators/SourceOperatorWatermarksTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,95 @@ | ||
/* | ||
Licensed to the Apache Software Foundation (ASF) under one | ||
or more contributor license agreements. See the NOTICE file | ||
distributed with this work for additional information | ||
regarding copyright ownership. The ASF licenses this file | ||
to you under the Apache License, Version 2.0 (the | ||
"License"); you may not use this file except in compliance | ||
with the License. You may obtain a copy of the License at | ||
http://www.apache.org/licenses/LICENSE-2.0 | ||
Unless required by applicable law or agreed to in writing, software | ||
distributed under the License is distributed on an "AS IS" BASIS, | ||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
See the License for the specific language governing permissions and | ||
limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.streaming.api.operators; | ||
|
||
import org.apache.flink.api.common.eventtime.WatermarkStrategy; | ||
import org.apache.flink.api.connector.source.mocks.MockSourceSplit; | ||
import org.apache.flink.streaming.api.operators.source.CollectingDataOutput; | ||
import org.apache.flink.streaming.api.watermark.Watermark; | ||
import org.apache.flink.streaming.runtime.io.DataInputStatus; | ||
|
||
import org.junit.jupiter.api.AfterEach; | ||
import org.junit.jupiter.api.BeforeEach; | ||
import org.junit.jupiter.api.Test; | ||
|
||
import javax.annotation.Nullable; | ||
|
||
import java.util.ArrayList; | ||
import java.util.List; | ||
|
||
import static org.assertj.core.api.Assertions.assertThat; | ||
|
||
/** Unit test for {@link SourceOperator} watermark alignment. */ | ||
@SuppressWarnings("serial") | ||
class SourceOperatorWatermarksTest { | ||
|
||
@Nullable private SourceOperatorTestContext context; | ||
@Nullable private SourceOperator<Integer, MockSourceSplit> operator; | ||
|
||
@BeforeEach | ||
void setup() throws Exception { | ||
context = | ||
new SourceOperatorTestContext( | ||
false, | ||
true, | ||
WatermarkStrategy.forGenerator( | ||
ctx -> | ||
new SourceOperatorAlignmentTest | ||
.PunctuatedGenerator()) | ||
.withTimestampAssigner((r, t) -> r)); | ||
operator = context.getOperator(); | ||
} | ||
|
||
@AfterEach | ||
void tearDown() throws Exception { | ||
context.close(); | ||
context = null; | ||
operator = null; | ||
} | ||
|
||
@Test | ||
void testWatermark() throws Exception { | ||
List<MockSourceSplit> initialSplits = new ArrayList<>(); | ||
initialSplits.add(new MockSourceSplit(0, 1000).addRecord(1042).addRecord(1044)); | ||
initialSplits.add(new MockSourceSplit(1, 1000).addRecord(42).addRecord(44)); | ||
operator.initializeState(context.createStateContext(initialSplits)); | ||
operator.open(); | ||
|
||
CollectingDataOutput<Integer> actualOutput = new CollectingDataOutput<>(); | ||
|
||
// after emitting first element from first split, there can not be watermark emitted, as | ||
// watermark from the other split is still unknown. | ||
assertThat(operator.emitNext(actualOutput)).isEqualTo(DataInputStatus.MORE_AVAILABLE); | ||
assertNoWatermarks(actualOutput); | ||
|
||
// after emitting two more elements (in this order: [1042, 1044, 42] but order doesn't | ||
// matter for this test), three in total, watermark 42 can be finally emitted | ||
assertThat(operator.emitNext(actualOutput)).isEqualTo(DataInputStatus.MORE_AVAILABLE); | ||
assertThat(operator.emitNext(actualOutput)).isEqualTo(DataInputStatus.MORE_AVAILABLE); | ||
assertWatermark(actualOutput, new Watermark(42)); | ||
} | ||
|
||
private static void assertNoWatermarks(CollectingDataOutput<Integer> actualOutput) { | ||
assertThat(actualOutput.getEvents()).noneMatch(element -> element instanceof Watermark); | ||
} | ||
|
||
private void assertWatermark(CollectingDataOutput<Integer> actualOutput, Watermark watermark) { | ||
assertThat(actualOutput.getEvents()).containsOnlyOnce(watermark); | ||
} | ||
} |