zhuzhurk commented on a change in pull request #12375:
URL: https://github.com/apache/flink/pull/12375#discussion_r437966502



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/BulkSlotProviderImpl.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.api.common.time.Time;
+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.SystemClock;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+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 PhysicalSlotRequestBulkTracker slotRequestBulkTracker;
+
+       BulkSlotProviderImpl(final SlotSelectionStrategy slotSelectionStrategy, 
final SlotPool slotPool) {
+               this.slotSelectionStrategy = 
checkNotNull(slotSelectionStrategy);
+               this.slotPool = checkNotNull(slotPool);
+
+               this.slotRequestBulkTracker = new 
PhysicalSlotRequestBulkTracker(
+                       this::getAllSlotInfos,
+                       SystemClock.getInstance());
+
+               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 void cancelSlotRequest(SlotRequestId slotRequestId, Throwable 
cause) {
+               componentMainThreadExecutor.assertRunningInMainThread();
+
+               slotPool.releaseSlot(slotRequestId, cause);
+       }
+
+       @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).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 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 if 
(physicalSlotRequest.willSlotBeOccupiedIndefinitely()) {
+                       slotFuture = slotPool.requestNewAllocatedSlot(
+                               slotRequestId,
+                               resourceProfile,
+                               null);
+               } else {
+                       slotFuture = slotPool.requestNewAllocatedBatchSlot(
+                               slotRequestId,
+                               resourceProfile);
+               }

Review comment:
       Maybe not right now given that it is not required by this change.
   The only difference that `requestNewAllocatedSlot` and 
`requestNewAllocatedBatchSlot` make is whether the request will fail-fast on 
normal allocation failures (i.e. failures that are not 
`UnfulfillableSlotRequestException`).
   I think we can remove the method `requestNewAllocatedBatchSlot` and do not 
differentiate batch/streaming requests in `SlotPool` anymore once when we are 
to disable the fail-fast for streaming slot requests on normal allocation 
failures. This will however lead to behavior changes and will need a discussion 
in community ML in think.




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


Reply via email to