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



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkTracker.java
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.jobmaster.SlotInfo;
+import org.apache.flink.util.clock.Clock;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.Optional;
+import java.util.Set;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Tracks physical slot request bulks. Once a bulk is registered, a 
fulfill-ability check for it will be scheduled.
+ */
+class PhysicalSlotRequestBulkTracker {
+
+       private final Supplier<Set<SlotInfo>> slotsRetriever;
+
+       private final Clock clock;
+
+       private final Set<PhysicalSlotRequestBulk> slotRequestBulks;
+
+       PhysicalSlotRequestBulkTracker(final Supplier<Set<SlotInfo>> 
slotsRetriever, final Clock clock) {
+               this.slotsRetriever = checkNotNull(slotsRetriever);
+               this.clock = checkNotNull(clock);
+               this.slotRequestBulks = Collections.newSetFromMap(new 
IdentityHashMap<>());
+       }
+
+       void track(final PhysicalSlotRequestBulk bulk) {
+               slotRequestBulks.add(bulk);
+
+               bulk.markUnfulfillable(clock.relativeTimeMillis());
+       }
+
+       void untrack(final PhysicalSlotRequestBulk bulk) {
+               slotRequestBulks.remove(bulk);
+       }
+
+       @VisibleForTesting
+       boolean isTracked(final PhysicalSlotRequestBulk bulk) {
+               return slotRequestBulks.contains(bulk);
+       }
+
+       /**
+        * 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

Review comment:
       ```suggestion
         * @param slotRequestTimeout indicates how long a pending request can 
be unfulfillable
   ```

##########
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);
+               }
+
+               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(() -> {
+                       final PhysicalSlotRequestBulkTracker.TimeoutCheckResult 
result =
+                               
slotRequestBulkTracker.checkPhysicalSlotRequestBulkTimeout(slotRequestBulk, 
timeout);
+
+                       switch (result) {
+                               case PENDING:
+                                       //re-schedule the timeout check
+                                       
schedulePendingRequestBulkTimeoutCheck(slotRequestBulk, timeout);
+                                       break;
+                               case TIMEOUT:
+                                       timeoutSlotRequestBulk(slotRequestBulk);
+                                       break;
+                               default: // no action to take
+                       }
+               }, timeout.getSize(), timeout.getUnit());
+       }
+
+       private void timeoutSlotRequestBulk(final PhysicalSlotRequestBulk 
slotRequestBulk) {
+               final Exception cause = new TimeoutException("Slot request bulk 
is not fulfillable!");
+               // pending requests must be canceled first otherwise they might 
be fulfilled by
+               // allocated slots released from this bulk
+               for (SlotRequestId slotRequestId : 
slotRequestBulk.getPendingRequests().keySet()) {
+                       cancelSlotRequest(slotRequestId, cause);
+               }
+               for (SlotRequestId slotRequestId : 
slotRequestBulk.getFulfilledRequests().keySet()) {
+                       cancelSlotRequest(slotRequestId, cause);
+               }
+       }
+
+       private Set<SlotInfo> getAllSlotInfos() {
+               return Stream
+                       .concat(
+                               
slotPool.getAvailableSlotsInformation().stream(),
+                               
slotPool.getAllocatedSlotsInformation().stream())
+                       .collect(Collectors.toSet());
+       }

Review comment:
       I thought we could actually move 
`timeoutSlotRequestBulk/getAllSlotInfos` and pass `slotPool` to 
`PhysicalSlotRequestBulkTracker`. Would this simplify multithreading in tests?
   For multithreading, we would only need to check whether 
`BulkSlotProviderImpl` properly schedules `checkPhysicalSlotRequestBulkTimeout`.

##########
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:
       nit: maybe one more method `slotFuture = 
requestNewSlot(willSlotBeOccupiedIndefinitely, slotRequestId, resourceProfile)`?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkTracker.java
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.jobmaster.SlotInfo;
+import org.apache.flink.util.clock.Clock;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.Optional;
+import java.util.Set;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Tracks physical slot request bulks. Once a bulk is registered, a 
fulfill-ability check for it will be scheduled.
+ */
+class PhysicalSlotRequestBulkTracker {
+
+       private final Supplier<Set<SlotInfo>> slotsRetriever;
+
+       private final Clock clock;
+
+       private final Set<PhysicalSlotRequestBulk> slotRequestBulks;
+
+       PhysicalSlotRequestBulkTracker(final Supplier<Set<SlotInfo>> 
slotsRetriever, final Clock clock) {
+               this.slotsRetriever = checkNotNull(slotsRetriever);
+               this.clock = checkNotNull(clock);
+               this.slotRequestBulks = Collections.newSetFromMap(new 
IdentityHashMap<>());
+       }
+
+       void track(final PhysicalSlotRequestBulk bulk) {
+               slotRequestBulks.add(bulk);
+
+               bulk.markUnfulfillable(clock.relativeTimeMillis());
+       }
+
+       void untrack(final PhysicalSlotRequestBulk bulk) {
+               slotRequestBulks.remove(bulk);
+       }
+
+       @VisibleForTesting
+       boolean isTracked(final PhysicalSlotRequestBulk bulk) {
+               return slotRequestBulks.contains(bulk);
+       }
+
+       /**
+        * 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 result of the check, indicating the bulk is fulfilled, still 
pending, or timed out
+        */
+       TimeoutCheckResult checkPhysicalSlotRequestBulkTimeout(
+                       final PhysicalSlotRequestBulk slotRequestBulk,
+                       final Time slotRequestTimeout) {
+
+               checkState(isTracked(slotRequestBulk));
+
+               if (slotRequestBulk.getPendingRequests().isEmpty()) {
+                       return TimeoutCheckResult.FULFILLED;
+               }
+
+               final boolean fulfillable = 
isSlotRequestBulkFulfillable(slotRequestBulk, slotsRetriever);
+               if (fulfillable) {
+                       slotRequestBulk.markFulfillable();
+               } else {
+                       final long currentTimestamp = 
clock.relativeTimeMillis();
+
+                       slotRequestBulk.markUnfulfillable(currentTimestamp);
+
+                       final long unfulfillableSince = 
slotRequestBulk.getUnfulfillableSince();
+                       if (unfulfillableSince + 
slotRequestTimeout.toMilliseconds() <= currentTimestamp) {
+                               return TimeoutCheckResult.TIMEOUT;
+                       }
+               }
+
+               return TimeoutCheckResult.PENDING;
+       }
+
+       /**
+        * Returns whether the given bulk of slot requests are possible to be 
fulfilled at the same time
+        * with all the reusable slots in the slot pool. A reusable slot means 
the slot is available or
+        * will not be occupied indefinitely.
+        *
+        * @param slotRequestBulk bulk of slot requests to check
+        * @param slotsRetriever supplies slots to be used for the 
fulfill-ability check
+        * @return true if the slot requests are possible to be fulfilled, 
otherwise false
+        */
+       @VisibleForTesting
+       static boolean isSlotRequestBulkFulfillable(

Review comment:
       Why do we need to expose `isSlotRequestBulkFulfillable`?
   Could we not test the same with `checkPhysicalSlotRequestBulkTimeout` and 
e.g. `PhysicalSlotRequestBulk.isFulfillable`?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkTracker.java
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.jobmaster.SlotInfo;
+import org.apache.flink.util.clock.Clock;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.Optional;
+import java.util.Set;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Tracks physical slot request bulks. Once a bulk is registered, a 
fulfill-ability check for it will be scheduled.
+ */
+class PhysicalSlotRequestBulkTracker {
+
+       private final Supplier<Set<SlotInfo>> slotsRetriever;
+
+       private final Clock clock;
+
+       private final Set<PhysicalSlotRequestBulk> slotRequestBulks;
+
+       PhysicalSlotRequestBulkTracker(final Supplier<Set<SlotInfo>> 
slotsRetriever, final Clock clock) {
+               this.slotsRetriever = checkNotNull(slotsRetriever);
+               this.clock = checkNotNull(clock);
+               this.slotRequestBulks = Collections.newSetFromMap(new 
IdentityHashMap<>());
+       }
+
+       void track(final PhysicalSlotRequestBulk bulk) {
+               slotRequestBulks.add(bulk);
+
+               bulk.markUnfulfillable(clock.relativeTimeMillis());
+       }
+
+       void untrack(final PhysicalSlotRequestBulk bulk) {
+               slotRequestBulks.remove(bulk);
+       }
+
+       @VisibleForTesting
+       boolean isTracked(final PhysicalSlotRequestBulk bulk) {
+               return slotRequestBulks.contains(bulk);
+       }
+
+       /**
+        * Check the slot request bulk and timeout its requests if it has been 
unfilfillable for too long.

Review comment:
       ```suggestion
         * Check the slot request bulk and timeout its requests if it has been 
unfulfillable for too long.
   ```

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkTracker.java
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.jobmaster.SlotInfo;
+import org.apache.flink.util.clock.Clock;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.Optional;
+import java.util.Set;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Tracks physical slot request bulks. Once a bulk is registered, a 
fulfill-ability check for it will be scheduled.
+ */
+class PhysicalSlotRequestBulkTracker {
+
+       private final Supplier<Set<SlotInfo>> slotsRetriever;
+
+       private final Clock clock;
+
+       private final Set<PhysicalSlotRequestBulk> slotRequestBulks;

Review comment:
       Not sure I understand why we need 
`slotRequestBulks/track/untrack/isTracked`.
   Looks only for tests. Can we remove it?




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


Reply via email to