rmetzger commented on a change in pull request #15071:
URL: https://github.com/apache/flink/pull/15071#discussion_r586557432



##########
File path: 
flink-tests/src/test/java/org/apache/flink/test/scheduling/ReactiveModeITCase.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.scheduling;
+
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.configuration.ClusterOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.configuration.SchedulerExecutionMode;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.testutils.MiniClusterResource;
+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.RichSinkFunction;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.test.util.MiniClusterWithClientResource;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.concurrent.CountDownLatch;
+
+/** Tests for Reactive Mode (FLIP-159). */
+public class ReactiveModeITCase extends TestLogger {
+    private static final int NUMBER_SLOTS_PER_TASK_MANAGER = 2;
+    private static final int INITIAL_NUMBER_TASK_MANAGERS = 1;
+
+    @ClassRule
+    public static final MiniClusterResource MINI_CLUSTER_WITH_CLIENT_RESOURCE =
+            new MiniClusterWithClientResource(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setConfiguration(getReactiveModeConfiguration())
+                            
.setNumberTaskManagers(INITIAL_NUMBER_TASK_MANAGERS)
+                            
.setNumberSlotsPerTaskManager(NUMBER_SLOTS_PER_TASK_MANAGER)
+                            .build());
+
+    private static Configuration getReactiveModeConfiguration() {
+        final Configuration conf = new Configuration();
+
+        conf.set(JobManagerOptions.SCHEDULER, 
JobManagerOptions.SchedulerType.Adaptive);
+        conf.set(JobManagerOptions.SCHEDULER_MODE, 
SchedulerExecutionMode.REACTIVE);
+        conf.set(ClusterOptions.ENABLE_DECLARATIVE_RESOURCE_MANAGEMENT, true);
+
+        return conf;
+    }
+
+    @Test
+    public void testScaleUpAndDownWithMaxParallelism() throws Exception {
+        StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1); // we set parallelism to ensure it's overwritten
+        
env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 
0L));
+        final DataStream<String> input = env.addSource(new 
ParallelismTrackingSource());
+        // we set maxParallelism = 1 and assert it never exceeds it
+        input.addSink(new 
ParallelismTrackingSink<>()).getTransformation().setMaxParallelism(1);
+
+        
ParallelismTrackingSource.expectInstances(NUMBER_SLOTS_PER_TASK_MANAGER);
+        ParallelismTrackingSink.expectInstances(1);
+
+        env.executeAsync();
+
+        ParallelismTrackingSource.waitForInstances();
+        ParallelismTrackingSink.waitForInstances();
+        ParallelismTrackingSource.expectInstances(2 * 
NUMBER_SLOTS_PER_TASK_MANAGER);
+        ParallelismTrackingSink.expectInstances(1);
+
+        final MiniCluster miniCluster = 
MINI_CLUSTER_WITH_CLIENT_RESOURCE.getMiniCluster();
+
+        // add additional TaskManager
+        miniCluster.startTaskManager();
+
+        ParallelismTrackingSource.waitForInstances();
+        ParallelismTrackingSink.waitForInstances();
+        // prepare for and scale down
+        
ParallelismTrackingSource.expectInstances(NUMBER_SLOTS_PER_TASK_MANAGER);
+        ParallelismTrackingSink.expectInstances(1);
+
+        miniCluster.terminateTaskManager(0);
+
+        ParallelismTrackingSource.waitForInstances();
+        ParallelismTrackingSink.waitForInstances();
+    }
+
+    private static class ParallelismTrackingSource implements 
ParallelSourceFunction<String> {
+        private volatile boolean running = true;
+
+        private static CountDownLatch instances;
+
+        public static void expectInstances(int count) {
+            instances = new CountDownLatch(count);
+        }
+
+        public static void waitForInstances() throws InterruptedException {
+            instances.await();
+        }
+
+        @Override
+        public void run(SourceContext<String> ctx) throws Exception {
+            instances.countDown();
+            while (running) {
+                ctx.collect("test");

Review comment:
       This is just an explanation for my thinking, not a counter-argument: As 
far as I understand the legacy sources, the lock is only relevant when using 
stateful sources: There you would do the state update and emission of elements 
under the lock. Since this is a stateless source there should be no need to use 
the lock.
   See also: 
https://github.com/apache/flink/blob/master/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java#L39




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to