dawidwys commented on a change in pull request #18726: URL: https://github.com/apache/flink/pull/18726#discussion_r804679753
########## File path: flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/AlignedWatermarksITCase.java ########## @@ -0,0 +1,161 @@ +/* + * 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.connector.base.source.reader; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobSubmissionResult; +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.common.eventtime.WatermarkGenerator; +import org.apache.flink.api.common.eventtime.WatermarkOutput; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.connector.source.lib.NumberSequenceSource; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.testutils.AllCallbackWrapper; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Metric; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.test.util.MiniClusterWithClientExtension; +import org.apache.flink.testutils.logging.LoggerAuditingExtension; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.slf4j.event.Level; + +import java.time.Duration; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * An IT test to verify watermark alignment works. The idea is that we check the {@link + * MetricNames#WATERMARK_DRIFT} metric does not grow beyond the maximal configured drift. It can + * initially increase beyond that value, because of the update interval, but once established it + * should never increase any further, but gradually decrease to the configured threshold, as the + * slower source catches up. + */ +public class AlignedWatermarksITCase { + public static final String SLOW_SOURCE_NAME = "SlowNumberSequenceSource"; + public static final String FAST_SOURCE_NAME = "FastNumberSequenceSource"; + public static final int MAX_DRIFT = 100; + + @RegisterExtension + LoggerAuditingExtension loggerAuditingExtension = + new LoggerAuditingExtension(AlignedWatermarksITCase.class, Level.INFO); + + private static final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + + public static final MiniClusterWithClientExtension MINI_CLUSTER_RESOURCE = + new MiniClusterWithClientExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setConfiguration(reporter.addToConfiguration(new Configuration())) + .build()); + + @RegisterExtension + static AllCallbackWrapper<MiniClusterWithClientExtension> allWrapper = + new AllCallbackWrapper<>(MINI_CLUSTER_RESOURCE); + + @Test + public void testAlignment() throws Exception { + final JobGraph jobGraph = getJobGraph(); + final MiniCluster miniCluster = MINI_CLUSTER_RESOURCE.getMiniCluster(); + final CompletableFuture<JobSubmissionResult> submission = miniCluster.submitJob(jobGraph); + final JobID jobID = submission.get().getJobID(); + CommonTestUtils.waitForAllTaskRunning(miniCluster, jobID, false); + + long oldDrift = Long.MAX_VALUE; + do { + final Optional<Metric> drift = + reporter.findMetric( + jobID, FAST_SOURCE_NAME + ".*" + MetricNames.WATERMARK_DRIFT); + Thread.sleep(200); + + final Optional<Long> newDriftOptional = drift.map(m -> ((Gauge<Long>) m).getValue()); + if (newDriftOptional.isPresent()) { + final Long newDrift = newDriftOptional.get(); + assertThat(newDrift).isLessThanOrEqualTo(oldDrift); + oldDrift = newDrift; + } + } while (oldDrift >= MAX_DRIFT); + } + + private JobGraph getJobGraph() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.getConfig().setAutoWatermarkInterval(0L); + env.setParallelism(1); + + DataStream<Long> slowSource = + env.fromSource( + new NumberSequenceSource(0, Long.MAX_VALUE), + WatermarkStrategy.forGenerator(ctx -> new PunctuatedGenerator()) + .withWatermarkAlignment( + "group-1", Duration.ofMillis(MAX_DRIFT)) Review comment: I adjusted the parameters (just to note, had to adjust the MAX_DRIFT as well). On my machine it takes ~3s to run (without the setup), compared to ~7s before. Still it takes a couple of rounds of updates to catch up. -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org