azagrebin commented on a change in pull request #13284: URL: https://github.com/apache/flink/pull/13284#discussion_r480166064
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java ########## @@ -300,6 +305,32 @@ public int getNumberOfVertices() { return this.taskVertices.size(); } + public Set<SlotSharingGroup> getSlotSharingGroups() { + final Set<SlotSharingGroup> slotSharingGroups = new HashSet<>(); + for (JobVertex vertex : getVertices()) { + final SlotSharingGroup slotSharingGroup = vertex.getSlotSharingGroup(); + checkNotNull(slotSharingGroup); + + slotSharingGroups.add(slotSharingGroup); + } + return Collections.unmodifiableSet(slotSharingGroups); + } + + public Set<CoLocationGroupDesc> getCoLocationGroupDescriptors() { + final Set<CoLocationGroup> coLocationGroups = new HashSet<>(); + for (JobVertex vertex : getVertices()) { + CoLocationGroup coLocationGroup = vertex.getCoLocationGroup(); + if (coLocationGroup != null) { + coLocationGroups.add(coLocationGroup); + } + } + final Set<CoLocationGroupDesc> coLocationGroupDescs = coLocationGroups + .stream() + .map(CoLocationGroupDesc::from) + .collect(Collectors.toSet()); Review comment: ```suggestion final Set<CoLocationGroup> coLocationGroupDescs = new HashSet<>(); for (JobVertex vertex : getVertices()) { CoLocationGroup coLocationGroup = vertex.getCoLocationGroup(); if (coLocationGroup != null) { CoLocationGroupDesc coLocationGroupDesc = CoLocationGroupDesc.from(coLocationGroup) coLocationGroups.add(coLocationGroupDesc); } } ``` ########## 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: does the test need to wait for 5s? could it be 1s or even less? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionSlotAllocatorFactory.java ########## @@ -19,11 +19,39 @@ package org.apache.flink.runtime.scheduler; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroupDesc; +import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; + +import java.util.Set; +import java.util.function.Function; +import java.util.function.Supplier; + /** * Interface for {@link ExecutionSlotAllocator} factories. */ +@FunctionalInterface public interface ExecutionSlotAllocatorFactory { - ExecutionSlotAllocator createInstance(PreferredLocationsRetriever preferredLocationsRetriever); - + /** + * Instantiates the {@link ExecutionSlotAllocator}. + * + * @param preferredLocationsRetriever to retrieve preferred locations for each execution + * @param resourceProfileRetriever to retrieve required {@link ResourceProfile} for each execution + * @param priorAllocationIdRetriever to retrieve prior {@link AllocationID} for each execution + * @param topology scheduling topology of the job + * @param logicalSlotSharingGroupSupplier to supply all slot sharing groups in the job + * @param coLocationGroupSupplier to supply all co-location groups in the job + * @return The instantiated slot allocator + */ + ExecutionSlotAllocator createInstance( + PreferredLocationsRetriever preferredLocationsRetriever, + Function<ExecutionVertexID, ResourceProfile> resourceProfileRetriever, + Function<ExecutionVertexID, AllocationID> priorAllocationIdRetriever, + SchedulingTopology topology, + Supplier<Set<SlotSharingGroup>> logicalSlotSharingGroupSupplier, + Supplier<Set<CoLocationGroupDesc>> coLocationGroupSupplier); Review comment: do `logicalSlotSharingGroupSupplier` and `coLocationGroupSupplier` have to be `Supplier`s? ########## 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: what is the difference between `parallelism` 2 and 100 for the tests? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionSlotAllocatorFactory.java ########## @@ -19,11 +19,39 @@ package org.apache.flink.runtime.scheduler; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroupDesc; +import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; + +import java.util.Set; +import java.util.function.Function; +import java.util.function.Supplier; + /** * Interface for {@link ExecutionSlotAllocator} factories. */ +@FunctionalInterface public interface ExecutionSlotAllocatorFactory { - ExecutionSlotAllocator createInstance(PreferredLocationsRetriever preferredLocationsRetriever); - + /** + * Instantiates the {@link ExecutionSlotAllocator}. + * + * @param preferredLocationsRetriever to retrieve preferred locations for each execution + * @param resourceProfileRetriever to retrieve required {@link ResourceProfile} for each execution + * @param priorAllocationIdRetriever to retrieve prior {@link AllocationID} for each execution + * @param topology scheduling topology of the job + * @param logicalSlotSharingGroupSupplier to supply all slot sharing groups in the job + * @param coLocationGroupSupplier to supply all co-location groups in the job + * @return The instantiated slot allocator + */ + ExecutionSlotAllocator createInstance( + PreferredLocationsRetriever preferredLocationsRetriever, + Function<ExecutionVertexID, ResourceProfile> resourceProfileRetriever, + Function<ExecutionVertexID, AllocationID> priorAllocationIdRetriever, + SchedulingTopology topology, + Supplier<Set<SlotSharingGroup>> logicalSlotSharingGroupSupplier, + Supplier<Set<CoLocationGroupDesc>> coLocationGroupSupplier); Review comment: I would consider uniting these parameters into e.g. ``` interface ExecutionSlotAllocationContext implements PreferredLocationsRetriever { ResourceProfile getResourceProfile(ExecutionVertexID); AllocationID getPriorAllocationId(ExecutionVertexID); ... } ``` ---------------------------------------------------------------- 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