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



##########
File path: 
flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
##########
@@ -339,6 +340,23 @@
                                .list(
                                        text("'ng': new generation scheduler"))
                                .build());
+
+       /**
+        * Config parameter determining the scheduling strategy.
+        */
+       @Documentation.ExcludeFromDocumentation("User normally should not be 
expected to change this config.")
+       public static final ConfigOption<String> SCHEDULING_STRATEGY =

Review comment:
       I would consider using `ConfigOption.enumType`, like e.g. 
`state.backend.rocksdb.timer-service.factory`

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java
##########
@@ -160,7 +160,15 @@
                        failoverStrategy,
                        restartBackoffTimeStrategy);
                this.schedulingStrategy = 
schedulingStrategyFactory.createInstance(this, getSchedulingTopology());
-               this.executionSlotAllocator = 
checkNotNull(executionSlotAllocatorFactory).createInstance(getPreferredLocationsRetriever());
+
+               final ExecutionSlotAllocationContext slotAllocationContext = 
new ExecutionSlotAllocationContext(

Review comment:
       Maybe this way it is easier  for testing.
   I was more thinking about an interface `ExecutionSlotAllocationContext` 
implemented internally in `DefaultScheduler` by wrapping 
`preferredLocationsRetriever/jobGraph/EG/schedulingTopology`.

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
##########
@@ -400,16 +400,23 @@ public void 
testJobWithAnOccasionallyFailingSenderVertex() throws Exception {
                try (final MiniCluster miniCluster = new MiniCluster(cfg)) {
                        miniCluster.start();
 
+                       // putting sender and receiver vertex in the same slot 
sharing group is required
+                       // to ensure all senders can be deployed. Otherwise 
this case can fail if the
+                       // expected failing sender is not deployed.

Review comment:
       are they not in the same SSG by default?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java
##########
@@ -127,7 +133,33 @@ public SchedulerNG createInstance(
                        executionDeploymentTracker);
        }
 
-       private static DefaultSchedulerComponents 
createDefaultSchedulerComponents(
+       @VisibleForTesting

Review comment:
       I would consider moving `DefaultSchedulerComponents` with its factory 
method`createSchedulerComponents` into a separate file to avoid 
`VisibleForTesting`, then `DefaultSchedulerFactoryTest` can be 
`DefaultSchedulerComponentsTest`.

##########
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);

Review comment:
       ok, 5s may be not a big deal for one test.
   
   another idea could be to sleep/poll connected TMs from 
`MiniCluster#requestClusterOverview`
   to wait for all TMs connected before running the job




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