Myasuka commented on a change in pull request #19033: URL: https://github.com/apache/flink/pull/19033#discussion_r829757672
########## File path: flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescaleCheckpointManuallyITCase.java ########## @@ -0,0 +1,314 @@ +/* + * 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.checkpointing; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.StateBackendOptions; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.state.KeyGroupRangeAssignment; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.CheckpointConfig; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.checkpointing.utils.RescalingTestUtils; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.util.TestUtils; + +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CountDownLatch; + +import static org.apache.flink.test.util.TestUtils.submitJobAndWaitForResult; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +/** Test checkpoint rescaling for incremental rocksdb. */ +public class RescaleCheckpointManuallyITCase { + + private static final int NUM_TASK_MANAGERS = 2; + private static final int SLOTS_PER_TASK_MANAGER = 2; + + private static MiniClusterWithClientResource cluster; + private File checkpointDir; + + @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Before + public void setup() throws Exception { + Configuration config = new Configuration(); + + checkpointDir = temporaryFolder.newFolder(); + + config.setString(StateBackendOptions.STATE_BACKEND, "rocksdb"); + config.setString( + CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString()); + config.setBoolean(CheckpointingOptions.INCREMENTAL_CHECKPOINTS, true); + + cluster = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setConfiguration(config) + .setNumberTaskManagers(NUM_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER) + .build()); + cluster.before(); + } + + @Test + public void testCheckpointRescalingInKeyedState() throws Exception { + testCheckpointRescalingKeyedState(false, false); + } + + @Test + public void testCheckpointRescalingOutKeyedState() throws Exception { + testCheckpointRescalingKeyedState(true, false); + } + + @Test + public void testCheckpointRescalingInKeyedStateDerivedMaxParallelism() throws Exception { + testCheckpointRescalingKeyedState(false, true); + } + + @Test + public void testCheckpointRescalingOutKeyedStateDerivedMaxParallelism() throws Exception { + testCheckpointRescalingKeyedState(true, true); + } Review comment: I think tests of rescale in and out is enough here. ########## File path: flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescaleCheckpointManuallyITCase.java ########## @@ -0,0 +1,314 @@ +/* + * 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.checkpointing; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.StateBackendOptions; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.state.KeyGroupRangeAssignment; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.CheckpointConfig; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.checkpointing.utils.RescalingTestUtils; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.util.TestUtils; + +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CountDownLatch; + +import static org.apache.flink.test.util.TestUtils.submitJobAndWaitForResult; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +/** Test checkpoint rescaling for incremental rocksdb. */ +public class RescaleCheckpointManuallyITCase { Review comment: This test should extend `TestLogger`. ########## File path: flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescaleCheckpointManuallyITCase.java ########## @@ -0,0 +1,314 @@ +/* + * 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.checkpointing; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.StateBackendOptions; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.state.KeyGroupRangeAssignment; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.CheckpointConfig; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.checkpointing.utils.RescalingTestUtils; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.util.TestUtils; + +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CountDownLatch; + +import static org.apache.flink.test.util.TestUtils.submitJobAndWaitForResult; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +/** Test checkpoint rescaling for incremental rocksdb. */ +public class RescaleCheckpointManuallyITCase { + + private static final int NUM_TASK_MANAGERS = 2; + private static final int SLOTS_PER_TASK_MANAGER = 2; + + private static MiniClusterWithClientResource cluster; + private File checkpointDir; + + @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Before + public void setup() throws Exception { + Configuration config = new Configuration(); + + checkpointDir = temporaryFolder.newFolder(); + + config.setString(StateBackendOptions.STATE_BACKEND, "rocksdb"); + config.setString( + CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString()); + config.setBoolean(CheckpointingOptions.INCREMENTAL_CHECKPOINTS, true); + + cluster = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setConfiguration(config) + .setNumberTaskManagers(NUM_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER) + .build()); + cluster.before(); + } + + @Test + public void testCheckpointRescalingInKeyedState() throws Exception { + testCheckpointRescalingKeyedState(false, false); + } + + @Test + public void testCheckpointRescalingOutKeyedState() throws Exception { + testCheckpointRescalingKeyedState(true, false); + } + + @Test + public void testCheckpointRescalingInKeyedStateDerivedMaxParallelism() throws Exception { + testCheckpointRescalingKeyedState(false, true); + } + + @Test + public void testCheckpointRescalingOutKeyedStateDerivedMaxParallelism() throws Exception { + testCheckpointRescalingKeyedState(true, true); + } + + /** + * Tests that a job with purely keyed state can be restarted from a checkpoint with a different + * parallelism. + */ + public void testCheckpointRescalingKeyedState(boolean scaleOut, boolean deriveMaxParallelism) + throws Exception { + final int numberKeys = 42; + final int numberElements = 1000; + final int numberElements2 = 500; + final int parallelism = scaleOut ? 3 : 4; + final int parallelism2 = scaleOut ? 4 : 3; + final int maxParallelism = 13; + + cluster.before(); + + ClusterClient<?> client = cluster.getClusterClient(); + String checkpointPath = + runJobAndGetCheckpoint( + numberKeys, + numberElements, + parallelism, + maxParallelism, + client, + checkpointDir); + + assertNotNull(checkpointPath); + + int restoreMaxParallelism = + deriveMaxParallelism ? JobVertex.MAX_PARALLELISM_DEFAULT : maxParallelism; + + restoreAndAssert( + parallelism2, + restoreMaxParallelism, + maxParallelism, + numberKeys, + numberElements2, + numberElements + numberElements2, + client, + checkpointPath); + } + + private static String runJobAndGetCheckpoint( + int numberKeys, + int numberElements, + int parallelism, + int maxParallelism, + ClusterClient<?> client, + File checkpointDir) + throws Exception { + try { + JobGraph jobGraph = + createJobGraphWithKeyedState( + parallelism, maxParallelism, numberKeys, numberElements, false, 100); + NotifyingDefiniteKeySource.countDownLatch = new CountDownLatch(parallelism); + client.submitJob(jobGraph).get(); + NotifyingDefiniteKeySource.countDownLatch.await(); + + RescalingTestUtils.SubtaskIndexFlatMapper.workCompletedLatch.await(); + + // verify the current state + Set<Tuple2<Integer, Integer>> actualResult = + RescalingTestUtils.CollectionSink.getElementsSet(); + + Set<Tuple2<Integer, Integer>> expectedResult = new HashSet<>(); + + for (int key = 0; key < numberKeys; key++) { + int keyGroupIndex = KeyGroupRangeAssignment.assignToKeyGroup(key, maxParallelism); + expectedResult.add( + Tuple2.of( + KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup( + maxParallelism, parallelism, keyGroupIndex), + numberElements * key)); + } + + assertEquals(expectedResult, actualResult); + NotifyingDefiniteKeySource.countDownLatch.await(); + + waitUntilExternalizedCheckpointCreated(checkpointDir); + client.cancel(jobGraph.getJobID()).get(); + TestUtils.waitUntilCanceled(jobGraph.getJobID(), client); + return TestUtils.getMostRecentCompletedCheckpoint(checkpointDir).getAbsolutePath(); + } finally { + RescalingTestUtils.CollectionSink.clearElementsSet(); + } + } + + private void restoreAndAssert( + int restoreParallelism, + int restoreMaxParallelism, + int maxParallelismBefore, + int numberKeys, + int numberElements, + int numberElementsExpect, + ClusterClient<?> client, + String restorePath) + throws Exception { + try { + + JobGraph scaledJobGraph = + createJobGraphWithKeyedState( + restoreParallelism, + restoreMaxParallelism, + numberKeys, + numberElements, + true, + 100); + + scaledJobGraph.setSavepointRestoreSettings( + SavepointRestoreSettings.forPath(restorePath)); + + submitJobAndWaitForResult(client, scaledJobGraph, getClass().getClassLoader()); + + Set<Tuple2<Integer, Integer>> actualResult2 = + RescalingTestUtils.CollectionSink.getElementsSet(); + + Set<Tuple2<Integer, Integer>> expectedResult2 = new HashSet<>(); + + for (int key = 0; key < numberKeys; key++) { + int keyGroupIndex = + KeyGroupRangeAssignment.assignToKeyGroup(key, maxParallelismBefore); + expectedResult2.add( + Tuple2.of( + KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup( + maxParallelismBefore, restoreParallelism, keyGroupIndex), + key * numberElementsExpect)); + } + assertEquals(expectedResult2, actualResult2); + } finally { + RescalingTestUtils.CollectionSink.clearElementsSet(); + } + } + + private static void waitUntilExternalizedCheckpointCreated(File checkpointDir) + throws InterruptedException, IOException { + while (true) { + Thread.sleep(50); + Optional<File> externalizedCheckpoint = + TestUtils.getMostRecentCompletedCheckpointMaybe(checkpointDir); + if (externalizedCheckpoint.isPresent()) { + break; + } + } + } Review comment: We can place this helper in `TestUtils` so that both `ResumeCheckpointManuallyITCase` and `RescaleCheckpointManuallyITCase` could use. ########## File path: flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/RescalingTestUtils.java ########## @@ -0,0 +1,171 @@ +/* + * 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.checkpointing.utils; + +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.api.common.state.CheckpointListener; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.util.Collector; + +import java.util.Collections; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; + +/** Test utilities for rescaling. */ +public class RescalingTestUtils { + + /** A parallel source with definite keys. */ + public static class DefiniteKeySource extends RichParallelSourceFunction<Integer> { + + private static final long serialVersionUID = -400066323594122516L; + + private final int numberKeys; + private final int numberElements; + private final boolean terminateAfterEmission; + + protected int counter = 0; + + private boolean running = true; + + public DefiniteKeySource( + int numberKeys, int numberElements, boolean terminateAfterEmission) { + this.numberKeys = numberKeys; + this.numberElements = numberElements; + this.terminateAfterEmission = terminateAfterEmission; + } + + @Override + public void run(SourceContext<Integer> ctx) throws Exception { + final Object lock = ctx.getCheckpointLock(); + final int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); + while (running) { + + if (counter < numberElements) { + synchronized (lock) { + for (int value = subtaskIndex; + value < numberKeys; + value += getRuntimeContext().getNumberOfParallelSubtasks()) { + ctx.collect(value); + } + counter++; + } + } else { + if (terminateAfterEmission) { + running = false; + } else { + Thread.sleep(100); + } + } + } + } + + @Override + public void cancel() { + running = false; + } + } + + /** A flatMapper with the index of subtask. */ + public static class SubtaskIndexFlatMapper + extends RichFlatMapFunction<Integer, Tuple2<Integer, Integer>> + implements CheckpointedFunction, CheckpointListener { + + private static final long serialVersionUID = 5273172591283191348L; + + public static CountDownLatch workCompletedLatch = new CountDownLatch(1); + private static AtomicInteger completedCheckpointNum; Review comment: There is no need to make `completedCheckpointNum` as static. ########## File path: flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/RescalingTestUtils.java ########## @@ -0,0 +1,171 @@ +/* + * 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.checkpointing.utils; + +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.api.common.state.CheckpointListener; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.util.Collector; + +import java.util.Collections; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; + +/** Test utilities for rescaling. */ +public class RescalingTestUtils { + + /** A parallel source with definite keys. */ + public static class DefiniteKeySource extends RichParallelSourceFunction<Integer> { + + private static final long serialVersionUID = -400066323594122516L; + + private final int numberKeys; + private final int numberElements; + private final boolean terminateAfterEmission; + + protected int counter = 0; + + private boolean running = true; + + public DefiniteKeySource( + int numberKeys, int numberElements, boolean terminateAfterEmission) { + this.numberKeys = numberKeys; + this.numberElements = numberElements; + this.terminateAfterEmission = terminateAfterEmission; + } + + @Override + public void run(SourceContext<Integer> ctx) throws Exception { + final Object lock = ctx.getCheckpointLock(); + final int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); + while (running) { + + if (counter < numberElements) { + synchronized (lock) { + for (int value = subtaskIndex; + value < numberKeys; + value += getRuntimeContext().getNumberOfParallelSubtasks()) { + ctx.collect(value); + } + counter++; + } + } else { + if (terminateAfterEmission) { + running = false; + } else { + Thread.sleep(100); + } + } + } + } + + @Override + public void cancel() { + running = false; + } + } + + /** A flatMapper with the index of subtask. */ + public static class SubtaskIndexFlatMapper + extends RichFlatMapFunction<Integer, Tuple2<Integer, Integer>> + implements CheckpointedFunction, CheckpointListener { + + private static final long serialVersionUID = 5273172591283191348L; + + public static CountDownLatch workCompletedLatch = new CountDownLatch(1); + private static AtomicInteger completedCheckpointNum; + + private transient ValueState<Integer> counter; + private transient ValueState<Integer> sum; + + private final int numberElements; + + public SubtaskIndexFlatMapper(int numberElements) { + this.numberElements = numberElements; + this.completedCheckpointNum = new AtomicInteger(); + } + + @Override + public void flatMap(Integer value, Collector<Tuple2<Integer, Integer>> out) + throws Exception { + + int count = counter.value() + 1; + counter.update(count); + + int s = sum.value() + value; + sum.update(s); + + if (count % numberElements == 0) { + out.collect(Tuple2.of(getRuntimeContext().getIndexOfThisSubtask(), s)); + workCompletedLatch.countDown(); + } + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + // all managed, nothing to do. + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + counter = + context.getKeyedStateStore() + .getState(new ValueStateDescriptor<>("counter", Integer.class, 0)); + sum = + context.getKeyedStateStore() + .getState(new ValueStateDescriptor<>("sum", Integer.class, 0)); + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + completedCheckpointNum.getAndIncrement(); + } + } + + /** Sink for collecting results into a collection. */ + public static class CollectionSink<IN> implements SinkFunction<IN> { + + private static Set<Object> elements = + Collections.newSetFromMap(new ConcurrentHashMap<Object, Boolean>()); Review comment: ```suggestion private static final Set<Object> elements = Collections.newSetFromMap(new ConcurrentHashMap<>()); ``` ########## File path: flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java ########## @@ -139,4 +142,11 @@ private static boolean hasMetadata(Path file) { return false; // should never happen } } + + public static void waitUntilCanceled(JobID jobId, ClusterClient<?> client) Review comment: I think `waitUntilJobCanceled` looks better. -- 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