zhuzhurk commented on a change in pull request #12256: URL: https://github.com/apache/flink/pull/12256#discussion_r431025011
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/OneSlotPerExecutionSlotAllocator.java ########## @@ -0,0 +1,260 @@ +/* + * 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.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.FutureUtils; +import org.apache.flink.runtime.executiongraph.SlotProviderStrategy; +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.PhysicalSlot; +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.util.FlinkException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.function.Function; +import java.util.stream.Collectors; + +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. + */ +public class OneSlotPerExecutionSlotAllocator extends DefaultExecutionSlotAllocator { + + private static final Logger LOG = LoggerFactory.getLogger(OneSlotPerExecutionSlotAllocator.class); + + private final SlotOwner slotOwner; + + public OneSlotPerExecutionSlotAllocator( + final SlotProviderStrategy slotProviderStrategy, + final InputsLocationsRetriever inputsLocationsRetriever) { + super(slotProviderStrategy, inputsLocationsRetriever); + + this.slotOwner = new OneSlotPerExecutionSlotAllocatorSlotOwner(); + } + + @Override + public List<SlotExecutionVertexAssignment> allocateSlotsFor( + final List<ExecutionVertexSchedulingRequirements> executionVertexSchedulingRequirements) { + + validateSchedulingRequirements(executionVertexSchedulingRequirements); + + validateNoCoLocationConstraint(executionVertexSchedulingRequirements); + + final Set<ExecutionVertexID> allExecutionVertexIds = executionVertexSchedulingRequirements.stream() + .map(ExecutionVertexSchedulingRequirements::getExecutionVertexId) + .collect(Collectors.toSet()); + + final Map<ExecutionVertexID, SlotRequestId> executionVertexSlotRequestIds = + generateExecutionVertexSlotRequestIds(allExecutionVertexIds); + + final List<SlotExecutionVertexAssignment> slotExecutionVertexAssignments = + createSlotExecutionVertexAssignments(executionVertexSchedulingRequirements, executionVertexSlotRequestIds); + + final List<CompletableFuture<PhysicalSlotRequest>> physicalSlotRequestFutures = + createPhysicalSlotRequestFutures( + executionVertexSchedulingRequirements, + allExecutionVertexIds, + executionVertexSlotRequestIds); + + allocateSlotsForAssignments( + physicalSlotRequestFutures, + slotExecutionVertexAssignments, + executionVertexSlotRequestIds); + + return 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 Map<ExecutionVertexID, SlotRequestId> generateExecutionVertexSlotRequestIds( + final Set<ExecutionVertexID> allExecutionVertexIds) { + + final Map<ExecutionVertexID, SlotRequestId> executionVertexSlotRequestIds = new HashMap<>(); + for (ExecutionVertexID executionVertexId : allExecutionVertexIds) { + executionVertexSlotRequestIds.put(executionVertexId, new SlotRequestId()); + } + return executionVertexSlotRequestIds; + } + + private List<SlotExecutionVertexAssignment> createSlotExecutionVertexAssignments( + final List<ExecutionVertexSchedulingRequirements> executionVertexSchedulingRequirements, + final Map<ExecutionVertexID, SlotRequestId> executionVertexSlotRequestIds) { + + final List<SlotExecutionVertexAssignment> slotExecutionVertexAssignments = + new ArrayList<>(executionVertexSchedulingRequirements.size()); + for (ExecutionVertexSchedulingRequirements schedulingRequirements : executionVertexSchedulingRequirements) { + final ExecutionVertexID executionVertexId = schedulingRequirements.getExecutionVertexId(); + final SlotRequestId slotRequestId = executionVertexSlotRequestIds.get(executionVertexId); + + final SlotExecutionVertexAssignment slotExecutionVertexAssignment = + createAndRegisterSlotExecutionVertexAssignment( + executionVertexId, + new CompletableFuture<>(), + slotRequestId, + null); + slotExecutionVertexAssignments.add(slotExecutionVertexAssignment); + } + + return slotExecutionVertexAssignments; + } + + private List<CompletableFuture<PhysicalSlotRequest>> createPhysicalSlotRequestFutures( + final List<ExecutionVertexSchedulingRequirements> executionVertexSchedulingRequirements, + final Set<ExecutionVertexID> allExecutionVertexIds, + final Map<ExecutionVertexID, SlotRequestId> executionVertexSlotRequestIds) { + + 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); + + // use the task resource profile as the physical slot resource requirement since slot sharing is ignored here + final ResourceProfile physicalSlotResourceProfile = schedulingRequirements.getTaskResourceProfile(); + + final CompletableFuture<SlotProfile> slotProfileFuture = createSlotProfile( + schedulingRequirements, + allExecutionVertexIds, + physicalSlotResourceProfile, + 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, slotProviderStrategy.willSlotBeOccupiedIndefinitely()); + } + + private void allocateSlotsForAssignments( + final List<CompletableFuture<PhysicalSlotRequest>> physicalSlotRequestFutures, + final List<SlotExecutionVertexAssignment> slotExecutionVertexAssignments, + final Map<ExecutionVertexID, SlotRequestId> executionVertexSlotRequestIds) { + + final Map<ExecutionVertexID, SlotExecutionVertexAssignment> vertexToAssignmentMapping = + slotExecutionVertexAssignments.stream() + .collect(Collectors.toMap(SlotExecutionVertexAssignment::getExecutionVertexId, Function.identity())); + + final Map<SlotRequestId, ExecutionVertexID> slotToVertexMapping = new HashMap<>(); + for (Map.Entry<ExecutionVertexID, SlotRequestId> entry : executionVertexSlotRequestIds.entrySet()) { + slotToVertexMapping.put(entry.getValue(), entry.getKey()); + } + + FutureUtils.combineAll(physicalSlotRequestFutures) + .thenCompose(physicalSlotRequests -> slotProviderStrategy.allocatePhysicalSlots(physicalSlotRequests)) + .thenAccept(physicalSlotRequestResults -> { + for (PhysicalSlotRequest.Result result : physicalSlotRequestResults) { + final SlotRequestId slotRequestId = result.getSlotRequestId(); + final ExecutionVertexID executionVertexId = slotToVertexMapping.get(slotRequestId); + try { + final LogicalSlot logicalSlot = allocateSingleLogicalSlotFromPhysicalSlot( + slotRequestId, + result.getPhysicalSlot()); + vertexToAssignmentMapping.get(executionVertexId).getLogicalSlotFuture().complete(logicalSlot); + } catch (Exception ex) { + throw new CompletionException(ex); + } + } + }) + .exceptionally(ex -> { + executionVertexSlotRequestIds.values().stream() + .forEach(slotRequestId -> { + final ExecutionVertexID executionVertexId = slotToVertexMapping.get(slotRequestId); + vertexToAssignmentMapping.get(executionVertexId).getLogicalSlotFuture().completeExceptionally(ex); + }); + return null; + }); + } + + private LogicalSlot allocateSingleLogicalSlotFromPhysicalSlot( + final SlotRequestId slotRequestId, + final PhysicalSlot physicalSlot) throws FlinkException { + + final SingleLogicalSlot singleTaskSlot = new SingleLogicalSlot( + slotRequestId, + physicalSlot, + null, + Locality.UNKNOWN, + slotOwner); + + if (physicalSlot.tryAssignPayload(singleTaskSlot)) { + return singleTaskSlot; + } else { + throw new FlinkException("Could not assign payload to allocated slot " + physicalSlot.getAllocationId() + '.'); Review comment: Good point. I think it is not supposed to happen with the new thread mode of JM. I will replace it with a state check as a sanity check. ---------------------------------------------------------------- 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]
