codenohup commented on code in PR #25731: URL: https://github.com/apache/flink/pull/25731#discussion_r1901090923
########## flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/WatermarkITCase.java: ########## @@ -0,0 +1,1064 @@ +/* + * 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.test.streaming.api.datastream; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.watermark.BoolWatermark; +import org.apache.flink.api.common.watermark.BoolWatermarkDeclaration; +import org.apache.flink.api.common.watermark.LongWatermark; +import org.apache.flink.api.common.watermark.LongWatermarkDeclaration; +import org.apache.flink.api.common.watermark.Watermark; +import org.apache.flink.api.common.watermark.WatermarkCombinationFunction; +import org.apache.flink.api.common.watermark.WatermarkCombinationPolicy; +import org.apache.flink.api.common.watermark.WatermarkDeclaration; +import org.apache.flink.api.common.watermark.WatermarkDeclarations; +import org.apache.flink.api.common.watermark.WatermarkHandlingResult; +import org.apache.flink.api.common.watermark.WatermarkHandlingStrategy; +import org.apache.flink.api.connector.dsv2.DataStreamV2SourceUtils; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.lib.NumberSequenceSource; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceReader; +import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.datastream.api.ExecutionEnvironment; +import org.apache.flink.datastream.api.common.Collector; +import org.apache.flink.datastream.api.context.NonPartitionedContext; +import org.apache.flink.datastream.api.context.PartitionedContext; +import org.apache.flink.datastream.api.function.OneInputStreamProcessFunction; +import org.apache.flink.datastream.api.stream.NonKeyedPartitionStream.ProcessConfigurableAndNonKeyedPartitionStream; +import org.apache.flink.datastream.impl.ExecutionEnvironmentImpl; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.minicluster.TestingMiniCluster; +import org.apache.flink.runtime.minicluster.TestingMiniClusterConfiguration; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.streaming.runtime.watermark.AlignableBoolWatermarkDeclaration; +import org.apache.flink.util.function.SupplierWithException; + +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.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; +import java.util.stream.LongStream; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * IT test case for {@link Watermark}. It will test the following behaviors of the generalized + * watermark: (1) The aligned watermark can block upstream. (2) The long watermark combines max/min + * correctly. (3) The bool watermark combines and/or correctly. (4) The watermark combiner will wait + * for all channels and then combine if the {@link + * WatermarkCombinationPolicy#isCombineWaitForAllChannels()} returns true. (5) The operator does not + * send watermarks when the {@link OneInputStreamProcessFunction#onWatermark} returns {@link + * WatermarkHandlingResult#POLL}. (6) The operator does not send watermarks when the {@link + * WatermarkHandlingStrategy} is set to IGNORE and the {@link + * OneInputStreamProcessFunction#onWatermark} returns {@link WatermarkHandlingResult#PEEK}. (7) The + * source operator can declare and emit watermarks. + * + * <p>We design a test job for these test cases, the test job has four operators, each with a + * parallelism of 2, and the shuffle edges are all to all. The job details are as follows: + * SourceOperator (Operator1) -> ProcessOperator(Operator2) -> ProcessOperator(Operator3) -> + * ProcessOperator(Operator4). Operator2 will declare watermarks and send watermark to downstream. + * Operator3 will receive watermarks from the upstream Operator2 and process them according to + * different test cases. Operator4 will receive or not receive watermarks from upstream Operator3. + */ +public class WatermarkITCase { + + private static final String DEFAULT_WATERMARK_IDENTIFIER = "default"; + + /** Parallelism of all operators. */ + private static final int DEFAULT_PARALLELISM = 2; + + private MiniCluster flinkCluster; + + /** + * The source operator will emit integer data within a range of 0 to {@code + * NUMBER_KEYS}(exclusive). + */ + private static final int NUMBER_KEYS = 100; + + public void startMiniCluster() throws Exception { + TestingMiniClusterConfiguration miniClusterConfiguration = + TestingMiniClusterConfiguration.newBuilder() + .setNumTaskManagers(3) + .setNumSlotsPerTaskManager(4) + .build(); + flinkCluster = TestingMiniCluster.newBuilder(miniClusterConfiguration).build(); + flinkCluster.start(); + } + + @BeforeEach + void before() throws Exception { + startMiniCluster(); + } + + @AfterEach + void after() throws Exception { + Operator2ProcessFunction.clear(); + Operator3ProcessFunction.clear(); + Operator4ProcessFunction.clear(); + + if (flinkCluster != null) { + flinkCluster.close(); + flinkCluster = null; + } + } + + /** + * Test aligned watermark can block upstream. In this test case, Operator2 will declare the + * aligned watermark. We will block Operator2's subtasks first and unblock them step by step, + * and each subtask will emit a watermark with value {@code true} after being unblocked. The + * Operator3 should receive only one combined and aligned watermark with value {@code true} + * after both the Operator2's subtask are unblocked. + */ + @Test + public void testAlignedWatermarkBlockUpstream() throws Exception { Review Comment: Hi @reswqa, I think I've already addressed this by unblocking Operator2's subtask 0. It should send records if it is not blocked. I also checked the received records in Operator3 after unblock Operator2's subtask 0, and they should be empty, which would demonstrate that the upstream Operator2 is indeed blocked. -- 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]
