azagrebin commented on a change in pull request #12256: URL: https://github.com/apache/flink/pull/12256#discussion_r438659428
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/OneSlotPerExecutionSlotAllocator.java ########## @@ -0,0 +1,240 @@ +/* + * 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.runtime.scheduler; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.SlotProfile; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.jobmanager.scheduler.Locality; +import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.jobmaster.SlotOwner; +import org.apache.flink.runtime.jobmaster.SlotRequestId; +import org.apache.flink.runtime.jobmaster.slotpool.BulkSlotProvider; +import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest; +import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot; +import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.util.FlinkException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * This slot allocator will request one physical slot for each single execution vertex. + * The slots will be requested in bulks so that the {@link SlotProvider} can check + * whether this bulk of slot requests can be fulfilled at the same time. + * It has several limitations: + * + * <p>1. Slot sharing will be ignored. + * + * <p>2. Co-location constraints are not allowed. + * + * <p>3. Intra-bulk input location preferences will be ignored. + */ +class OneSlotPerExecutionSlotAllocator extends AbstractExecutionSlotAllocator { + + private static final Logger LOG = LoggerFactory.getLogger(OneSlotPerExecutionSlotAllocator.class); + + private final BulkSlotProvider slotProvider; + + private final SlotOwner slotOwner; + + private final boolean slotWillBeOccupiedIndefinitely; + + private final Time allocationTimeout; + + OneSlotPerExecutionSlotAllocator( + final BulkSlotProvider slotProvider, + final PreferredLocationsRetriever preferredLocationsRetriever, + final boolean slotWillBeOccupiedIndefinitely, + final Time allocationTimeout) { + + super(preferredLocationsRetriever); + + this.slotProvider = checkNotNull(slotProvider); + this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely; + this.allocationTimeout = checkNotNull(allocationTimeout); + + this.slotOwner = new OneSlotPerExecutionSlotAllocatorSlotOwner(); + } + + @Override + public List<SlotExecutionVertexAssignment> allocateSlotsFor( + final List<ExecutionVertexSchedulingRequirements> executionVertexSchedulingRequirements) { + + validateSchedulingRequirements(executionVertexSchedulingRequirements); + + validateNoCoLocationConstraint(executionVertexSchedulingRequirements); + + final List<SlotExecutionVertexAssignment> slotExecutionVertexAssignments = + createSlotExecutionVertexAssignments(executionVertexSchedulingRequirements); + + final Map<ExecutionVertexID, SlotRequestId> executionVertexSlotRequestIds = slotExecutionVertexAssignments + .stream() + .collect( + Collectors.toMap( + SlotExecutionVertexAssignment::getExecutionVertexId, + SlotExecutionVertexAssignment::getSlotRequestId)); + + final List<CompletableFuture<PhysicalSlotRequest>> physicalSlotRequestFutures = + createPhysicalSlotRequestFutures( + executionVertexSchedulingRequirements, + executionVertexSlotRequestIds); + + allocateSlotsForAssignments( + physicalSlotRequestFutures, + slotExecutionVertexAssignments); + + return Collections.unmodifiableList(slotExecutionVertexAssignments); + } + + private static void validateNoCoLocationConstraint( + final Collection<ExecutionVertexSchedulingRequirements> schedulingRequirements) { + + final boolean hasCoLocationConstraint = schedulingRequirements.stream() + .anyMatch(r -> r.getCoLocationConstraint() != null); + checkState( + !hasCoLocationConstraint, + "Jobs with co-location constraints are not allowed to run with pipelined region scheduling strategy."); + } + + private List<SlotExecutionVertexAssignment> createSlotExecutionVertexAssignments( + final List<ExecutionVertexSchedulingRequirements> executionVertexSchedulingRequirements) { + + final List<SlotExecutionVertexAssignment> slotExecutionVertexAssignments = + new ArrayList<>(executionVertexSchedulingRequirements.size()); + for (ExecutionVertexSchedulingRequirements schedulingRequirements : executionVertexSchedulingRequirements) { + final ExecutionVertexID executionVertexId = schedulingRequirements.getExecutionVertexId(); + + final SlotExecutionVertexAssignment slotExecutionVertexAssignment = + createAndRegisterSlotExecutionVertexAssignment( + executionVertexId, + new CompletableFuture<>(), + (slotRequestId, throwable) -> slotProvider.cancelSlotRequest(slotRequestId, throwable)); + slotExecutionVertexAssignments.add(slotExecutionVertexAssignment); + } + + return slotExecutionVertexAssignments; + } + + private List<CompletableFuture<PhysicalSlotRequest>> createPhysicalSlotRequestFutures( + final List<ExecutionVertexSchedulingRequirements> executionVertexSchedulingRequirements, + final Map<ExecutionVertexID, SlotRequestId> executionVertexSlotRequestIds) { + + final Set<ExecutionVertexID> allExecutionVertexIds = executionVertexSchedulingRequirements.stream() + .map(ExecutionVertexSchedulingRequirements::getExecutionVertexId) + .collect(Collectors.toSet()); + + final Set<AllocationID> allPreviousAllocationIds = + computeAllPriorAllocationIds(executionVertexSchedulingRequirements); + + final List<CompletableFuture<PhysicalSlotRequest>> physicalSlotRequestFutures = + new ArrayList<>(executionVertexSchedulingRequirements.size()); + for (ExecutionVertexSchedulingRequirements schedulingRequirements : executionVertexSchedulingRequirements) { + final ExecutionVertexID executionVertexId = schedulingRequirements.getExecutionVertexId(); + final SlotRequestId slotRequestId = executionVertexSlotRequestIds.get(executionVertexId); + + LOG.debug("Allocate slot with id {} for execution {}", slotRequestId, executionVertexId); + + final CompletableFuture<Collection<TaskManagerLocation>> preferredLocationsFuture = + calculatePreferredLocations(executionVertexId, allExecutionVertexIds); + + // use the task resource profile as the physical slot resource requirement since slot sharing is ignored + final CompletableFuture<SlotProfile> slotProfileFuture = preferredLocationsFuture.thenApply( + preferredLocations -> + SlotProfile.priorAllocation( + schedulingRequirements.getTaskResourceProfile(), + schedulingRequirements.getTaskResourceProfile(), + preferredLocations, + Collections.singletonList(schedulingRequirements.getPreviousAllocationId()), + allPreviousAllocationIds)); + + final CompletableFuture<PhysicalSlotRequest> physicalSlotRequestFuture = + slotProfileFuture.thenApply( + slotProfile -> createPhysicalSlotRequest(slotRequestId, slotProfile)); + physicalSlotRequestFutures.add(physicalSlotRequestFuture); + } + + return physicalSlotRequestFutures; + } + + private PhysicalSlotRequest createPhysicalSlotRequest( + final SlotRequestId slotRequestId, + final SlotProfile slotProfile) { + return new PhysicalSlotRequest(slotRequestId, slotProfile, slotWillBeOccupiedIndefinitely); + } + + private void allocateSlotsForAssignments( + final List<CompletableFuture<PhysicalSlotRequest>> physicalSlotRequestFutures, + final List<SlotExecutionVertexAssignment> slotExecutionVertexAssignments) { + + final Map<SlotRequestId, SlotExecutionVertexAssignment> requestToAssignmentMapping = + slotExecutionVertexAssignments.stream() + .collect(Collectors.toMap(SlotExecutionVertexAssignment::getSlotRequestId, Function.identity())); + + FutureUtils.combineAll(physicalSlotRequestFutures) + .thenCompose(physicalSlotRequests -> slotProvider.allocatePhysicalSlots(physicalSlotRequests, allocationTimeout)) + .thenAccept(physicalSlotRequestResults -> { + for (PhysicalSlotRequest.Result result : physicalSlotRequestResults) { + final SlotRequestId slotRequestId = result.getSlotRequestId(); + final SlotExecutionVertexAssignment assignment = requestToAssignmentMapping.get(slotRequestId); + + checkState(assignment != null); + + final LogicalSlot logicalSlot = SingleLogicalSlot.allocateFromPhysicalSlot( + slotRequestId, + result.getPhysicalSlot(), + Locality.UNKNOWN, + slotOwner, + slotWillBeOccupiedIndefinitely); + assignment.getLogicalSlotFuture().complete(logicalSlot); + } + }) + .exceptionally(ex -> { + slotExecutionVertexAssignments.forEach( + assignment -> assignment.getLogicalSlotFuture().completeExceptionally(ex)); + return null; + }); + } + + private class OneSlotPerExecutionSlotAllocatorSlotOwner implements SlotOwner { + + @Override + public void returnLogicalSlot(LogicalSlot logicalSlot) { + slotProvider.cancelSlotRequest( + logicalSlot.getSlotRequestId(), + new FlinkException("Slot is being returned to OneSlotPerExecutionSlotAllocator.")); + } + } Review comment: ```suggestion @Override public void returnLogicalSlot(LogicalSlot logicalSlot) { slotProvider.cancelSlotRequest( logicalSlot.getSlotRequestId(), new FlinkException("Slot is being returned to OneSlotPerExecutionSlotAllocator.")); } ``` Maybe it is just easier to implement `SlotOwner` in `OneSlotPerExecutionSlotAllocator` and pass `this` to `SingleLogicalSlot.allocateFromPhysicalSlot`? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocator.java ########## @@ -36,41 +35,28 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; -import java.util.Objects; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.stream.Collectors; import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.apache.flink.util.Preconditions.checkState; /** * Default {@link ExecutionSlotAllocator} which will use {@link SlotProvider} to allocate slots and * keep the unfulfilled requests for further cancellation. */ -public class DefaultExecutionSlotAllocator implements ExecutionSlotAllocator { +public class DefaultExecutionSlotAllocator extends AbstractExecutionSlotAllocator { private static final Logger LOG = LoggerFactory.getLogger(DefaultExecutionSlotAllocator.class); - /** - * Store the uncompleted slot assignments. - */ - private final Map<ExecutionVertexID, SlotExecutionVertexAssignment> pendingSlotAssignments; - private final SlotProviderStrategy slotProviderStrategy; - private final PreferredLocationsRetriever preferredLocationsRetriever; - public DefaultExecutionSlotAllocator( final SlotProviderStrategy slotProviderStrategy, final PreferredLocationsRetriever preferredLocationsRetriever) { - this.slotProviderStrategy = checkNotNull(slotProviderStrategy); - this.preferredLocationsRetriever = checkNotNull(preferredLocationsRetriever); - pendingSlotAssignments = new HashMap<>(); + super(preferredLocationsRetriever); Review comment: nit: constructor can be package-private ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/AbstractExecutionSlotAllocator.java ########## @@ -0,0 +1,119 @@ +/* + * 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.runtime.scheduler; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.jobmaster.SlotRequestId; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.function.BiConsumer; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Base class for all {@link ExecutionSlotAllocator}. It is responsible to allocate slots for tasks and + * keep the unfulfilled slot requests for further cancellation. + */ +abstract class AbstractExecutionSlotAllocator implements ExecutionSlotAllocator { + + private final Map<ExecutionVertexID, SlotExecutionVertexAssignment> pendingSlotAssignments; + + private final PreferredLocationsRetriever preferredLocationsRetriever; + + protected AbstractExecutionSlotAllocator(final PreferredLocationsRetriever preferredLocationsRetriever) { + this.preferredLocationsRetriever = checkNotNull(preferredLocationsRetriever); + this.pendingSlotAssignments = new HashMap<>(); + } + + @Override + public void cancel(final ExecutionVertexID executionVertexId) { + final SlotExecutionVertexAssignment slotExecutionVertexAssignment = pendingSlotAssignments.get(executionVertexId); + if (slotExecutionVertexAssignment != null) { + slotExecutionVertexAssignment.getLogicalSlotFuture().cancel(false); + } + } + + protected void validateSchedulingRequirements( Review comment: ```suggestion void validateSchedulingRequirements( ``` nit: not sure, why we need `protected` for methods in `AbstractExecutionSlotAllocator` if it is used only within the package ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/OneSlotPerExecutionSlotAllocatorTest.java ########## @@ -0,0 +1,305 @@ +/* + * 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.runtime.scheduler; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.clusterframework.types.SlotProfile; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.instance.SlotSharingGroupId; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint; +import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; +import org.apache.flink.runtime.jobmaster.SlotRequestId; +import org.apache.flink.runtime.jobmaster.slotpool.BulkSlotProvider; +import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.util.TestLogger; + +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotTestUtils.createPhysicalSlot; +import static org.apache.flink.runtime.scheduler.ExecutionSlotAllocatorTestUtils.createSchedulingRequirements; +import static org.apache.flink.runtime.scheduler.ExecutionSlotAllocatorTestUtils.findSlotAssignmentByExecutionVertexId; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; +import static org.junit.Assert.assertThat; + +/** + * Tests for {@link OneSlotPerExecutionSlotAllocator}. + */ +public class OneSlotPerExecutionSlotAllocatorTest extends TestLogger { + + private TestingBulkSlotProvider slotProvider; + + @Before + public void setUp() throws Exception { + slotProvider = new TestingBulkSlotProvider(); + } + + @Test + public void testSucceededSlotAllocation() { + final ExecutionSlotAllocator executionSlotAllocator = createExecutionSlotAllocator(); + + final ExecutionVertexID executionVertexID = new ExecutionVertexID(new JobVertexID(), 0); + final List<ExecutionVertexSchedulingRequirements> schedulingRequirements = + createSchedulingRequirements(executionVertexID); + + final Collection<SlotExecutionVertexAssignment> slotExecutionVertexAssignments = + executionSlotAllocator.allocateSlotsFor(schedulingRequirements); + + assertThat(slotExecutionVertexAssignments, hasSize(1)); + + final SlotExecutionVertexAssignment slotAssignment = + findSlotAssignmentByExecutionVertexId(executionVertexID, slotExecutionVertexAssignments); + + assertThat(slotAssignment.getExecutionVertexId(), equalTo(executionVertexID)); + assertThat(slotAssignment.getLogicalSlotFuture().isDone(), is(true)); + assertThat(slotAssignment.getLogicalSlotFuture().isCompletedExceptionally(), is(false)); + } + + @Test + public void testFailedSlotAllocation() { + final OneSlotPerExecutionSlotAllocator executionSlotAllocator = createExecutionSlotAllocator(); + + final ExecutionVertexID executionVertexID = new ExecutionVertexID(new JobVertexID(), 0); + final List<ExecutionVertexSchedulingRequirements> schedulingRequirements = + createSchedulingRequirements(executionVertexID); + + slotProvider.forceFailingSlotAllocation(); + final Collection<SlotExecutionVertexAssignment> slotExecutionVertexAssignments = + executionSlotAllocator.allocateSlotsFor(schedulingRequirements); + + final SlotExecutionVertexAssignment slotAssignment = + findSlotAssignmentByExecutionVertexId(executionVertexID, slotExecutionVertexAssignments); + + assertThat(slotAssignment.getLogicalSlotFuture().isCompletedExceptionally(), is(true)); + assertThat(executionSlotAllocator.getPendingSlotAssignments().keySet(), hasSize(0)); + assertThat(slotProvider.getCancelledSlotRequestIds(), contains(slotAssignment.getSlotRequestId())); + } + + @Test + public void testInterBulkInputLocationPreferencesAreRespected() { + final ExecutionVertexID producerId = new ExecutionVertexID(new JobVertexID(), 0); + final ExecutionVertexID consumerId = new ExecutionVertexID(new JobVertexID(), 0); + + final TestingInputsLocationsRetriever inputsLocationsRetriever = new TestingInputsLocationsRetriever.Builder() + .connectConsumerToProducer(consumerId, producerId) + .build(); + + final ExecutionSlotAllocator executionSlotAllocator = createExecutionSlotAllocator( + new TestingStateLocationRetriever(), + inputsLocationsRetriever); + + inputsLocationsRetriever.markScheduled(producerId); + final List<ExecutionVertexSchedulingRequirements> schedulingRequirementsForProducer = + createSchedulingRequirements(producerId); + final Collection<SlotExecutionVertexAssignment> slotExecutionVertexAssignmentsForProducer = + executionSlotAllocator.allocateSlotsFor(schedulingRequirementsForProducer); + final SlotExecutionVertexAssignment producerSlotAssignment = + findSlotAssignmentByExecutionVertexId(producerId, slotExecutionVertexAssignmentsForProducer); + + assertThat(producerSlotAssignment.getLogicalSlotFuture().isDone(), is(true)); + + inputsLocationsRetriever.markScheduled(consumerId); + final List<ExecutionVertexSchedulingRequirements> schedulingRequirementsForConsumer = + createSchedulingRequirements(consumerId); + final Collection<SlotExecutionVertexAssignment> slotExecutionVertexAssignmentsForConsumer = + executionSlotAllocator.allocateSlotsFor(schedulingRequirementsForConsumer); + final SlotExecutionVertexAssignment consumerSlotAssignment = + findSlotAssignmentByExecutionVertexId(consumerId, slotExecutionVertexAssignmentsForConsumer); + + assertThat(consumerSlotAssignment.getLogicalSlotFuture().isDone(), is(false)); + + inputsLocationsRetriever.assignTaskManagerLocation(producerId); + + assertThat(consumerSlotAssignment.getLogicalSlotFuture().isDone(), is(true)); + } + + @Test + public void testIntraBulkInputLocationPreferencesDoNotBlockAllocation() { + final ExecutionVertexID producerId = new ExecutionVertexID(new JobVertexID(), 0); + final ExecutionVertexID consumerId = new ExecutionVertexID(new JobVertexID(), 0); + + final TestingInputsLocationsRetriever inputsLocationsRetriever = new TestingInputsLocationsRetriever.Builder() + .connectConsumerToProducer(consumerId, producerId) + .build(); + + final ExecutionSlotAllocator executionSlotAllocator = createExecutionSlotAllocator( + new TestingStateLocationRetriever(), + inputsLocationsRetriever); + + inputsLocationsRetriever.markScheduled(producerId); + inputsLocationsRetriever.markScheduled(consumerId); + + final List<ExecutionVertexSchedulingRequirements> schedulingRequirements = + createSchedulingRequirements(producerId, consumerId); + final Collection<SlotExecutionVertexAssignment> slotExecutionVertexAssignments = + executionSlotAllocator.allocateSlotsFor(schedulingRequirements); + + assertThat(slotExecutionVertexAssignments, hasSize(2)); + + final SlotExecutionVertexAssignment producerSlotAssignment = + findSlotAssignmentByExecutionVertexId(producerId, slotExecutionVertexAssignments); + final SlotExecutionVertexAssignment consumerSlotAssignment = + findSlotAssignmentByExecutionVertexId(consumerId, slotExecutionVertexAssignments); + + assertThat(producerSlotAssignment.getLogicalSlotFuture().isDone(), is(true)); + assertThat(consumerSlotAssignment.getLogicalSlotFuture().isDone(), is(true)); + } + + @Test + public void testCreatedSlotRequests() { + final ExecutionVertexID executionVertexId = new ExecutionVertexID(new JobVertexID(), 0); + final AllocationID allocationId = new AllocationID(); + final SlotSharingGroupId sharingGroupId = new SlotSharingGroupId(); + final ResourceProfile taskResourceProfile = ResourceProfile.fromResources(0.5, 250); + final ResourceProfile physicalSlotResourceProfile = ResourceProfile.fromResources(1.0, 300); + final TaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); + + final TestingStateLocationRetriever stateLocationRetriever = new TestingStateLocationRetriever(); + stateLocationRetriever.setStateLocation(executionVertexId, taskManagerLocation); + + final ExecutionSlotAllocator executionSlotAllocator = createExecutionSlotAllocator( + stateLocationRetriever, + new TestingInputsLocationsRetriever.Builder().build()); + + final List<ExecutionVertexSchedulingRequirements> schedulingRequirements = Arrays.asList( Review comment: ```suggestion final List<ExecutionVertexSchedulingRequirements> schedulingRequirements = Collections.singletonList( ``` ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocator.java ########## @@ -77,22 +75,20 @@ public DefaultExecutionSlotAllocator( LOG.debug("Allocate slot with id {} for execution {}", slotRequestId, executionVertexId); - CompletableFuture<LogicalSlot> slotFuture = calculatePreferredLocations( - executionVertexId, - Collections.emptySet()).thenCompose( - (Collection<TaskManagerLocation> preferredLocations) -> - slotProviderStrategy.allocateSlot( - slotRequestId, - new ScheduledUnit( - executionVertexId, - slotSharingGroupId, - schedulingRequirements.getCoLocationConstraint()), - SlotProfile.priorAllocation( - schedulingRequirements.getTaskResourceProfile(), - schedulingRequirements.getPhysicalSlotResourceProfile(), - preferredLocations, - Collections.singletonList(schedulingRequirements.getPreviousAllocationId()), - allPreviousAllocationIds))); + final CompletableFuture<SlotProfile> slotProfileFuture = getSlotProfileFuture( + schedulingRequirements, + schedulingRequirements.getPhysicalSlotResourceProfile(), + Collections.emptySet(), + allPreviousAllocationIds); + + final CompletableFuture<LogicalSlot> slotFuture = slotProfileFuture.thenCompose( + slotProfile -> slotProviderStrategy.allocateSlot( + slotRequestId, + new ScheduledUnit( + executionVertexId, + slotSharingGroupId, + schedulingRequirements.getCoLocationConstraint()), + slotProfile)); Review comment: nit, it would be nice to have: ``` private CompletableFuture<LogicalSlot> allocateSlot( Set<AllocationID> allPreviousAllocationIds, ExecutionVertexSchedulingRequirements schedulingRequirements) { final ExecutionVertexID executionVertexId = schedulingRequirements.getExecutionVertexId(); final SlotRequestId slotRequestId = new SlotRequestId(); LOG.debug("Allocate slot with id {} for execution {}", slotRequestId, executionVertexId); final CompletableFuture<SlotProfile> slotProfileFuture = getSlotProfileFuture( schedulingRequirements, schedulingRequirements.getPhysicalSlotResourceProfile(), Collections.emptySet(), allPreviousAllocationIds); return slotProfileFuture.thenCompose( slotProfile -> slotProviderStrategy.allocateSlot( slotRequestId, new ScheduledUnit( executionVertexId, schedulingRequirements.getSlotSharingGroupId(), schedulingRequirements.getCoLocationConstraint()), slotProfile)); } ``` ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/AbstractExecutionSlotAllocator.java ########## @@ -94,7 +96,26 @@ protected SlotExecutionVertexAssignment createAndRegisterSlotExecutionVertexAssi return slotExecutionVertexAssignment; } - protected CompletableFuture<Collection<TaskManagerLocation>> calculatePreferredLocations( + protected CompletableFuture<SlotProfile> getSlotProfileFuture( + final ExecutionVertexSchedulingRequirements schedulingRequirements, + final ResourceProfile physicalSlotResourceProfile, + final Set<ExecutionVertexID> producersToIgnore, + final Set<AllocationID> allPreviousAllocationIds) { + + final CompletableFuture<Collection<TaskManagerLocation>> preferredLocationsFuture = + calculatePreferredLocations(schedulingRequirements.getExecutionVertexId(), producersToIgnore); + + return preferredLocationsFuture.thenApply( + preferredLocations -> + SlotProfile.priorAllocation( + schedulingRequirements.getTaskResourceProfile(), + physicalSlotResourceProfile, + preferredLocations, + Collections.singletonList(schedulingRequirements.getPreviousAllocationId()), + allPreviousAllocationIds)); + } + + private CompletableFuture<Collection<TaskManagerLocation>> calculatePreferredLocations( Review comment: If it is a private method now, I guess we do not need it? ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/OneSlotPerExecutionSlotAllocatorTest.java ########## @@ -0,0 +1,305 @@ +/* + * 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.runtime.scheduler; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.clusterframework.types.SlotProfile; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.instance.SlotSharingGroupId; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint; +import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; +import org.apache.flink.runtime.jobmaster.SlotRequestId; +import org.apache.flink.runtime.jobmaster.slotpool.BulkSlotProvider; +import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.util.TestLogger; + +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotTestUtils.createPhysicalSlot; +import static org.apache.flink.runtime.scheduler.ExecutionSlotAllocatorTestUtils.createSchedulingRequirements; +import static org.apache.flink.runtime.scheduler.ExecutionSlotAllocatorTestUtils.findSlotAssignmentByExecutionVertexId; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; +import static org.junit.Assert.assertThat; + +/** + * Tests for {@link OneSlotPerExecutionSlotAllocator}. + */ +public class OneSlotPerExecutionSlotAllocatorTest extends TestLogger { + + private TestingBulkSlotProvider slotProvider; + + @Before + public void setUp() throws Exception { + slotProvider = new TestingBulkSlotProvider(); + } + + @Test + public void testSucceededSlotAllocation() { + final ExecutionSlotAllocator executionSlotAllocator = createExecutionSlotAllocator(); + + final ExecutionVertexID executionVertexID = new ExecutionVertexID(new JobVertexID(), 0); + final List<ExecutionVertexSchedulingRequirements> schedulingRequirements = + createSchedulingRequirements(executionVertexID); + + final Collection<SlotExecutionVertexAssignment> slotExecutionVertexAssignments = + executionSlotAllocator.allocateSlotsFor(schedulingRequirements); + + assertThat(slotExecutionVertexAssignments, hasSize(1)); + + final SlotExecutionVertexAssignment slotAssignment = + findSlotAssignmentByExecutionVertexId(executionVertexID, slotExecutionVertexAssignments); + + assertThat(slotAssignment.getExecutionVertexId(), equalTo(executionVertexID)); + assertThat(slotAssignment.getLogicalSlotFuture().isDone(), is(true)); + assertThat(slotAssignment.getLogicalSlotFuture().isCompletedExceptionally(), is(false)); + } + + @Test + public void testFailedSlotAllocation() { + final OneSlotPerExecutionSlotAllocator executionSlotAllocator = createExecutionSlotAllocator(); + + final ExecutionVertexID executionVertexID = new ExecutionVertexID(new JobVertexID(), 0); + final List<ExecutionVertexSchedulingRequirements> schedulingRequirements = + createSchedulingRequirements(executionVertexID); + + slotProvider.forceFailingSlotAllocation(); + final Collection<SlotExecutionVertexAssignment> slotExecutionVertexAssignments = + executionSlotAllocator.allocateSlotsFor(schedulingRequirements); + + final SlotExecutionVertexAssignment slotAssignment = + findSlotAssignmentByExecutionVertexId(executionVertexID, slotExecutionVertexAssignments); + + assertThat(slotAssignment.getLogicalSlotFuture().isCompletedExceptionally(), is(true)); + assertThat(executionSlotAllocator.getPendingSlotAssignments().keySet(), hasSize(0)); + assertThat(slotProvider.getCancelledSlotRequestIds(), contains(slotAssignment.getSlotRequestId())); + } + + @Test + public void testInterBulkInputLocationPreferencesAreRespected() { + final ExecutionVertexID producerId = new ExecutionVertexID(new JobVertexID(), 0); + final ExecutionVertexID consumerId = new ExecutionVertexID(new JobVertexID(), 0); + + final TestingInputsLocationsRetriever inputsLocationsRetriever = new TestingInputsLocationsRetriever.Builder() + .connectConsumerToProducer(consumerId, producerId) + .build(); + + final ExecutionSlotAllocator executionSlotAllocator = createExecutionSlotAllocator( + new TestingStateLocationRetriever(), + inputsLocationsRetriever); + + inputsLocationsRetriever.markScheduled(producerId); + final List<ExecutionVertexSchedulingRequirements> schedulingRequirementsForProducer = + createSchedulingRequirements(producerId); + final Collection<SlotExecutionVertexAssignment> slotExecutionVertexAssignmentsForProducer = + executionSlotAllocator.allocateSlotsFor(schedulingRequirementsForProducer); + final SlotExecutionVertexAssignment producerSlotAssignment = + findSlotAssignmentByExecutionVertexId(producerId, slotExecutionVertexAssignmentsForProducer); + + assertThat(producerSlotAssignment.getLogicalSlotFuture().isDone(), is(true)); + + inputsLocationsRetriever.markScheduled(consumerId); + final List<ExecutionVertexSchedulingRequirements> schedulingRequirementsForConsumer = + createSchedulingRequirements(consumerId); + final Collection<SlotExecutionVertexAssignment> slotExecutionVertexAssignmentsForConsumer = + executionSlotAllocator.allocateSlotsFor(schedulingRequirementsForConsumer); + final SlotExecutionVertexAssignment consumerSlotAssignment = + findSlotAssignmentByExecutionVertexId(consumerId, slotExecutionVertexAssignmentsForConsumer); + + assertThat(consumerSlotAssignment.getLogicalSlotFuture().isDone(), is(false)); + + inputsLocationsRetriever.assignTaskManagerLocation(producerId); + + assertThat(consumerSlotAssignment.getLogicalSlotFuture().isDone(), is(true)); + } + + @Test + public void testIntraBulkInputLocationPreferencesDoNotBlockAllocation() { + final ExecutionVertexID producerId = new ExecutionVertexID(new JobVertexID(), 0); + final ExecutionVertexID consumerId = new ExecutionVertexID(new JobVertexID(), 0); + + final TestingInputsLocationsRetriever inputsLocationsRetriever = new TestingInputsLocationsRetriever.Builder() + .connectConsumerToProducer(consumerId, producerId) + .build(); + + final ExecutionSlotAllocator executionSlotAllocator = createExecutionSlotAllocator( + new TestingStateLocationRetriever(), + inputsLocationsRetriever); + + inputsLocationsRetriever.markScheduled(producerId); + inputsLocationsRetriever.markScheduled(consumerId); + + final List<ExecutionVertexSchedulingRequirements> schedulingRequirements = + createSchedulingRequirements(producerId, consumerId); + final Collection<SlotExecutionVertexAssignment> slotExecutionVertexAssignments = + executionSlotAllocator.allocateSlotsFor(schedulingRequirements); + + assertThat(slotExecutionVertexAssignments, hasSize(2)); + + final SlotExecutionVertexAssignment producerSlotAssignment = + findSlotAssignmentByExecutionVertexId(producerId, slotExecutionVertexAssignments); + final SlotExecutionVertexAssignment consumerSlotAssignment = + findSlotAssignmentByExecutionVertexId(consumerId, slotExecutionVertexAssignments); + + assertThat(producerSlotAssignment.getLogicalSlotFuture().isDone(), is(true)); + assertThat(consumerSlotAssignment.getLogicalSlotFuture().isDone(), is(true)); + } + + @Test + public void testCreatedSlotRequests() { + final ExecutionVertexID executionVertexId = new ExecutionVertexID(new JobVertexID(), 0); + final AllocationID allocationId = new AllocationID(); + final SlotSharingGroupId sharingGroupId = new SlotSharingGroupId(); + final ResourceProfile taskResourceProfile = ResourceProfile.fromResources(0.5, 250); + final ResourceProfile physicalSlotResourceProfile = ResourceProfile.fromResources(1.0, 300); + final TaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); + + final TestingStateLocationRetriever stateLocationRetriever = new TestingStateLocationRetriever(); + stateLocationRetriever.setStateLocation(executionVertexId, taskManagerLocation); + + final ExecutionSlotAllocator executionSlotAllocator = createExecutionSlotAllocator( + stateLocationRetriever, + new TestingInputsLocationsRetriever.Builder().build()); + + final List<ExecutionVertexSchedulingRequirements> schedulingRequirements = Arrays.asList( + new ExecutionVertexSchedulingRequirements.Builder() + .withExecutionVertexId(executionVertexId) + .withPreviousAllocationId(allocationId) + .withSlotSharingGroupId(sharingGroupId) + .withPhysicalSlotResourceProfile(physicalSlotResourceProfile) + .withTaskResourceProfile(taskResourceProfile) + .build() + ); + + executionSlotAllocator.allocateSlotsFor(schedulingRequirements); + assertThat(slotProvider.getSlotRequests(), hasSize(1)); + + final SlotProfile requestSlotProfile = slotProvider.getSlotRequests().iterator().next().getSlotProfile(); + + assertThat(requestSlotProfile.getPreferredAllocations(), contains(allocationId)); + assertThat(requestSlotProfile.getPreviousExecutionGraphAllocations(), contains(allocationId)); + assertThat(requestSlotProfile.getTaskResourceProfile(), equalTo(taskResourceProfile)); + assertThat(requestSlotProfile.getPreferredLocations(), contains(taskManagerLocation)); + // task resource profile is used instead of slot sharing group resource profile since slot sharing is ignored + assertThat(requestSlotProfile.getPhysicalSlotResourceProfile(), equalTo(taskResourceProfile)); + } + + @Test(expected = IllegalStateException.class) + public void testCoLocationConstraintThrowsException() { + final ExecutionSlotAllocator executionSlotAllocator = createExecutionSlotAllocator(); + + final CoLocationConstraint coLocationConstraint = new CoLocationGroup().getLocationConstraint(0); + final List<ExecutionVertexSchedulingRequirements> schedulingRequirements = Arrays.asList( Review comment: ```suggestion final List<ExecutionVertexSchedulingRequirements> schedulingRequirements = Collections.singletonList( ``` ---------------------------------------------------------------- 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]
