zhuzhurk commented on a change in pull request #13284:
URL: https://github.com/apache/flink/pull/13284#discussion_r482004118



##########
File path: 
flink-tests/src/test/java/org/apache/flink/test/scheduling/PipelinedRegionSchedulingITCase.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.JobID;
+import org.apache.flink.client.program.MiniClusterClient;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.ScheduleMode;
+import 
org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
+import org.apache.flink.runtime.jobmaster.JobResult;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.minicluster.MiniClusterConfiguration;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * IT case for pipelined region scheduling.
+ */
+public class PipelinedRegionSchedulingITCase extends TestLogger {
+
+       @Test
+       public void testSuccessWithSlotsNoFewerThanTheMaxRegionRequired() 
throws Exception {
+               final JobResult jobResult = executeSchedulingTest(2);
+               assertThat(jobResult.getSerializedThrowable().isPresent(), 
is(false));
+       }
+
+       @Test
+       public void testFailsOnInsufficientSlots() throws Exception {
+               final JobResult jobResult = executeSchedulingTest(1);
+               assertThat(jobResult.getSerializedThrowable().isPresent(), 
is(true));
+
+               final Throwable jobFailure = jobResult
+                       .getSerializedThrowable()
+                       .get()
+                       .deserializeError(ClassLoader.getSystemClassLoader());
+
+               final Optional<NoResourceAvailableException> cause = 
ExceptionUtils.findThrowable(
+                       jobFailure,
+                       NoResourceAvailableException.class);
+               assertThat(cause.isPresent(), is(true));
+               assertThat(cause.get().getMessage(), containsString("Slot 
request bulk is not fulfillable!"));
+       }
+
+       private JobResult executeSchedulingTest(int numSlots) throws Exception {
+               final Configuration configuration = new Configuration();
+               configuration.setString(RestOptions.BIND_PORT, "0");
+               configuration.setLong(JobManagerOptions.SLOT_REQUEST_TIMEOUT, 
5000L);
+
+               final MiniClusterConfiguration miniClusterConfiguration = new 
MiniClusterConfiguration.Builder()
+                       .setConfiguration(configuration)
+                       .setNumTaskManagers(1)
+                       .setNumSlotsPerTaskManager(numSlots)
+                       .build();
+
+               try (MiniCluster miniCluster = new 
MiniCluster(miniClusterConfiguration)) {
+                       miniCluster.start();
+
+                       final MiniClusterClient miniClusterClient = new 
MiniClusterClient(configuration, miniCluster);
+
+                       final JobGraph jobGraph = createJobGraph(100);

Review comment:
       I have improved the testing job to not pass with lazy-form-sources 
scheduling. The parallelism is decreased to 10.
   A parallelism > 8 is required here. This is because in lazy-form-sources 
scheduling, the downstream task will wait for all the upstream task to be 
assigned slots if the parallelism is very small. This will result in source 
tasks to be scheduled earlier than sink tasks, and the job will not deadlock 
due to this side effect.




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to