zhuzhurk commented on a change in pull request #12375: URL: https://github.com/apache/flink/pull/12375#discussion_r435194969
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/BulkSlotProviderImpl.java ########## @@ -0,0 +1,271 @@ +/* + * 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.jobmaster.slotpool; + +import org.apache.flink.annotation.VisibleForTesting; +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.jobmaster.SlotInfo; +import org.apache.flink.runtime.jobmaster.SlotRequestId; +import org.apache.flink.util.clock.Clock; +import org.apache.flink.util.clock.SystemClock; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Default implementation of {@link BulkSlotProvider}. + */ +class BulkSlotProviderImpl implements BulkSlotProvider { + + private static final Logger LOG = LoggerFactory.getLogger(BulkSlotProviderImpl.class); + + private ComponentMainThreadExecutor componentMainThreadExecutor; + + private final SlotSelectionStrategy slotSelectionStrategy; + + private final SlotPool slotPool; + + private final Clock clock; + + private final PhysicalSlotRequestBulkTracker slotRequestBulkTracker; + + BulkSlotProviderImpl(final SlotSelectionStrategy slotSelectionStrategy, final SlotPool slotPool) { + this(slotSelectionStrategy, slotPool, SystemClock.getInstance()); + } + + @VisibleForTesting + BulkSlotProviderImpl( + final SlotSelectionStrategy slotSelectionStrategy, + final SlotPool slotPool, + final Clock clock) { + + this.slotSelectionStrategy = checkNotNull(slotSelectionStrategy); + this.slotPool = checkNotNull(slotPool); + this.clock = checkNotNull(clock); + + this.slotRequestBulkTracker = new PhysicalSlotRequestBulkTracker(clock); + + this.componentMainThreadExecutor = new ComponentMainThreadExecutor.DummyComponentMainThreadExecutor( + "Scheduler is not initialized with proper main thread executor. " + + "Call to BulkSlotProvider.start(...) required."); + } + + @Override + public void start(final ComponentMainThreadExecutor mainThreadExecutor) { + this.componentMainThreadExecutor = mainThreadExecutor; + } + + @Override + public CompletableFuture<Collection<PhysicalSlotRequest.Result>> allocatePhysicalSlots( + final Collection<PhysicalSlotRequest> physicalSlotRequests, + final Time timeout) { + + componentMainThreadExecutor.assertRunningInMainThread(); + + LOG.debug("Received {} slot requests.", physicalSlotRequests.size()); + + final PhysicalSlotRequestBulk slotRequestBulk = new PhysicalSlotRequestBulk(physicalSlotRequests); + + final List<CompletableFuture<PhysicalSlotRequest.Result>> resultFutures = new ArrayList<>(physicalSlotRequests.size()); + for (PhysicalSlotRequest request : physicalSlotRequests) { + final CompletableFuture<PhysicalSlotRequest.Result> resultFuture = + allocatePhysicalSlot(request, timeout).thenApply(result -> { + slotRequestBulk.markRequestFulfilled( + result.getSlotRequestId(), + result.getPhysicalSlot().getAllocationId()); + + return result; + }); + resultFutures.add(resultFuture); + } + + slotRequestBulkTracker.track(slotRequestBulk); + schedulePendingRequestBulkTimeoutCheck(slotRequestBulk, timeout); + + return FutureUtils.combineAll(resultFutures) + .whenComplete((ignore, throwable) -> slotRequestBulkTracker.untrack(slotRequestBulk)); + } + + private CompletableFuture<PhysicalSlotRequest.Result> allocatePhysicalSlot( + final PhysicalSlotRequest physicalSlotRequest, + final Time timeout) { + + final SlotRequestId slotRequestId = physicalSlotRequest.getSlotRequestId(); + final SlotProfile slotProfile = physicalSlotRequest.getSlotProfile(); + final ResourceProfile resourceProfile = slotProfile.getPhysicalSlotResourceProfile(); + + LOG.debug("Received slot request [{}] with resource requirements: {}", slotRequestId, resourceProfile); + + final Optional<PhysicalSlot> availablePhysicalSlot = tryAllocateFromAvailable(slotRequestId, slotProfile); + + final CompletableFuture<PhysicalSlot> slotFuture; + if (availablePhysicalSlot.isPresent()) { + slotFuture = CompletableFuture.completedFuture(availablePhysicalSlot.get()); + } else { + slotFuture = slotPool.requestNewAllocatedSlotWithoutTimeout( + slotRequestId, + resourceProfile, + !physicalSlotRequest.willSlotBeOccupiedIndefinitely()); + } + + return slotFuture.thenApply(physicalSlot -> new PhysicalSlotRequest.Result(slotRequestId, physicalSlot)); + } + + private Optional<PhysicalSlot> tryAllocateFromAvailable( + final SlotRequestId slotRequestId, + final SlotProfile slotProfile) { + + final Collection<SlotSelectionStrategy.SlotInfoAndResources> slotInfoList = + slotPool.getAvailableSlotsInformation() + .stream() + .map(SlotSelectionStrategy.SlotInfoAndResources::fromSingleSlot) + .collect(Collectors.toList()); + + final Optional<SlotSelectionStrategy.SlotInfoAndLocality> selectedAvailableSlot = + slotSelectionStrategy.selectBestSlotForProfile(slotInfoList, slotProfile); + + return selectedAvailableSlot.flatMap( + slotInfoAndLocality -> slotPool.allocateAvailableSlot( + slotRequestId, + slotInfoAndLocality.getSlotInfo().getAllocationId()) + ); + } + + private void schedulePendingRequestBulkTimeoutCheck( + final PhysicalSlotRequestBulk slotRequestBulk, + final Time timeout) { + + componentMainThreadExecutor.schedule(() -> { + if (!checkPhysicalSlotRequestBulkTimeout(slotRequestBulk, timeout)) { + schedulePendingRequestBulkTimeoutCheck(slotRequestBulk, timeout); + } + }, timeout.getSize(), timeout.getUnit()); + } + + /** + * Check the slot request bulk and timeout its requests if it has been unfilfillable for too long. + * @param slotRequestBulk bulk of slot requests + * @param slotRequestTimeout indicates how long a pending request can be unfilfillable + * @return true if the slot request bulk is done or timed out, otherwise false + */ + @VisibleForTesting + protected boolean checkPhysicalSlotRequestBulkTimeout( Review comment: Sounds good to me. Let me take another look. ---------------------------------------------------------------- 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]
