azagrebin commented on a change in pull request #12375: URL: https://github.com/apache/flink/pull/12375#discussion_r431914289
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulk.java ########## @@ -0,0 +1,59 @@ +/* + * 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.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.jobmaster.SlotRequestId; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Represents a bulk of physical slot requests. + */ +public class PhysicalSlotRequestBulk { Review comment: ```suggestion class PhysicalSlotRequestBulk { ``` ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/BulkSlotAllocationTest.java ########## @@ -0,0 +1,370 @@ +/* + * 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.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.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.jobmanager.slots.TestingSlotOwner; +import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.jobmaster.SlotInfo; +import org.apache.flink.runtime.jobmaster.SlotRequestId; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.TestLogger; +import org.apache.flink.util.clock.ManualClock; + +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +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.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.flink.runtime.jobmaster.slotpool.AllocatedSlotOccupationTest.allocateSingleLogicalSlotFromPhysicalSlot; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +/** + * Tests whether bulk slot allocation works correctly. + */ +public class BulkSlotAllocationTest extends TestLogger { Review comment: ```suggestion public class BulkSlotProviderImplTest extends TestLogger { ``` ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/AllocatedSlotOccupationTest.java ########## @@ -0,0 +1,98 @@ +/* + * 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.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; +import org.apache.flink.runtime.jobmanager.scheduler.Locality; +import org.apache.flink.runtime.jobmanager.slots.TestingSlotOwner; +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.taskmanager.LocalTaskManagerLocation; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Tests whether the slot occupation state of {@link AllocatedSlot} is correctly. + */ +public class AllocatedSlotOccupationTest extends TestLogger { + + @Test + public void testSingleTaskOccupyingSlotIndefinitely() { + final PhysicalSlot physicalSlot = createPhysicalSlot(); + allocateSingleLogicalSlotFromPhysicalSlot(physicalSlot, true); + + assertTrue(physicalSlot.willBeOccupiedIndefinitely()); + } + + @Test + public void testSingleTaskNotOccupyingSlotIndefinitely() { + final PhysicalSlot physicalSlot = createPhysicalSlot(); + allocateSingleLogicalSlotFromPhysicalSlot(physicalSlot, false); + + assertFalse(physicalSlot.willBeOccupiedIndefinitely()); + } + + private static PhysicalSlot createPhysicalSlot() { + return new AllocatedSlot( + new AllocationID(), + new LocalTaskManagerLocation(), + 0, + ResourceProfile.ANY, + new SimpleAckingTaskManagerGateway()); + } + + static LogicalSlot allocateSingleLogicalSlotFromPhysicalSlot( + final PhysicalSlot physicalSlot, + final boolean slotWillBeOccupiedIndefinitely) { + + return allocateSingleLogicalSlotFromPhysicalSlot( + new SlotRequestId(), + physicalSlot, + new TestingSlotOwner(), + slotWillBeOccupiedIndefinitely); + } + + static LogicalSlot allocateSingleLogicalSlotFromPhysicalSlot( Review comment: nit: We have this logic in some places in production as well. Maybe, this could factored out into `SingleLogicalSlot#allocateFromPhysicalSlot`? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImpl.java ########## @@ -447,12 +452,40 @@ public void releaseSlot(@Nonnull SlotRequestId slotRequestId, @Nullable Throwabl componentMainThreadExecutor.assertRunningInMainThread(); + checkState(batchSlotTimeoutCheckEnabled, "batch slot timeout check is disabled unexpectedly."); + final PendingRequest pendingRequest = PendingRequest.createBatchRequest(slotRequestId, resourceProfile); return requestNewAllocatedSlotInternal(pendingRequest) .thenApply(Function.identity()); } + @Override + public CompletableFuture<PhysicalSlot> requestNewAllocatedSlotWithoutTimeout( Review comment: I am wondering whether it would be more clear just to add `disableBatchSlotTimeoutCheck` explicitly to the public interface. We could also make `timeout` `Nullable` in `requestNewAllocatedSlot` then `FutureUtils.orTimeout` can be conditional in `requestNewAllocatedSlot`. The we do not need `requestNewAllocatedSlotWithoutTimeout` and make any assumptions in it about `batchSlotTimeoutCheckEnabled`. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotProvider.java ########## @@ -91,6 +92,20 @@ allocationTimeout); } + /** + * Allocates a bulk of physical slots. The allocation will be completed + * normally only when all the requests are fulfilled. + * + * @param physicalSlotRequests requests for physical slots + * @param timeout indicating how long it is accepted that the slot requests can be unfulfillable + * @return future of the results of slot requests + */ + default CompletableFuture<Collection<PhysicalSlotRequest.Result>> allocatePhysicalSlots( Review comment: Maybe, `SlotProvider` could just extend `BulkSlotProvider`? ########## 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) { Review comment: What do you think about `BulkSlotProviderImpl#createWithSystemClock()`? ########## 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( Review comment: maybe no immediate action now, but I find it quite hard to judge about failures in SlotPoolImpl , especially for bulk allocation. With this new concept of fulfil-ability, maybe we should simplify SlotPoolImpl and let BulkSlotProviderImpl judge about how to interpret external request failures and when to fulfil the requests, it could be a follow-up. We might want to note that new slot allocation has failed but there are often chances that the pending request can be fulfilled by allocated slots. ########## 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: Generally, making methods `VisibleForTesting` looks like we have one more concern to test internally. Could we factor out this concern to test separately? What if we create a `PhysicalSlotRequestBulkTracker(requests, slotPool)` for each bulk by merging it with `PhysicalSlotRequestBulk`? or `PhysicalSlotRequestBulk` could have the `unfulfillableSinceTimestamp` internally. Not sure, why we need the existing `PhysicalSlotRequestBulkTracker` for _all_ bulk timestamps. All methods here, related to timeout/fulfilablity check, could also go into the `PhysicalSlotRequestBulkTracker`. `BulkSlotProviderImpl` could schedule `PhysicalSlotRequestBulkTracker#checkPhysicalSlotRequestBulkTimeout` for each bulk. `PhysicalSlotRequestBulkTracker#checkPhysicalSlotRequestBulkTimeout` could return FINISHED/FULFILLABLE/TIMEOUT. Then `PhysicalSlotRequestBulkTracker#checkPhysicalSlotRequestBulkTimeout` could be tested separately. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/AllocatedSlotOccupationTest.java ########## @@ -0,0 +1,84 @@ +/* + * 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.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; +import org.apache.flink.runtime.jobmanager.scheduler.Locality; +import org.apache.flink.runtime.jobmanager.slots.TestingSlotOwner; +import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.jobmaster.SlotRequestId; +import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Tests whether the slot occupation state of {@link AllocatedSlot} is correctly. + */ +public class AllocatedSlotOccupationTest extends TestLogger { + + @Test + public void testSingleTaskOccupyingSlotIndefinitely() { + final PhysicalSlot physicalSlot = createPhysicalSlot(); + allocateSingleLogicalSlotFromPhysicalSlot(physicalSlot, true); + + assertTrue(physicalSlot.willBeOccupiedIndefinitely()); Review comment: ```suggestion assertThat(physicalSlot.willBeOccupiedIndefinitely(), is(true)); ``` also in other places ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkTrackerTest.java ########## @@ -0,0 +1,82 @@ +/* + * 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.util.TestLogger; +import org.apache.flink.util.clock.ManualClock; + +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; +import java.util.concurrent.TimeUnit; + +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +/** + * Tests for {@link PhysicalSlotRequestBulkTracker}. + */ +public class PhysicalSlotRequestBulkTrackerTest extends TestLogger { + + private ManualClock clock = new ManualClock(); + + private PhysicalSlotRequestBulkTracker bulkTracker; + + @Before + public void setup() throws Exception { + bulkTracker = new PhysicalSlotRequestBulkTracker(clock); + } + + @Test + public void testBulkTracking() { + final PhysicalSlotRequestBulk bulk = new PhysicalSlotRequestBulk(Collections.emptyList()); + + assertThat(bulkTracker.isTracked(bulk), is(false)); + + bulkTracker.track(bulk); + + assertThat(bulkTracker.isTracked(bulk), is(true)); + assertThat(bulkTracker.getBulkUnfulfillableSince(bulk), is(clock.relativeTimeMillis())); + + bulkTracker.untrack(bulk); + + assertThat(bulkTracker.isTracked(bulk), is(false)); + } + + @Test + public void testUnfulfillableTimestampWillNotBeOverriddenByFollowingUnfulfillableTimestamp() { + final PhysicalSlotRequestBulk bulk = new PhysicalSlotRequestBulk(Collections.emptyList()); + bulkTracker.track(bulk); + + final long unfulfillableSince = clock.relativeTimeMillis(); + + assertThat(bulkTracker.getBulkUnfulfillableSince(bulk), is(unfulfillableSince)); + + clock.advanceTime(456, TimeUnit.MILLISECONDS); + bulkTracker.markBulkUnfulfillable(bulk, clock.relativeTimeMillis()); + + assertThat(bulkTracker.getBulkUnfulfillableSince(bulk), is(unfulfillableSince)); + + bulkTracker.markBulkFulfillable(bulk); + bulkTracker.markBulkUnfulfillable(bulk, clock.relativeTimeMillis()); + + assertThat(bulkTracker.getBulkUnfulfillableSince(bulk), is(clock.relativeTimeMillis())); Review comment: can this be a separate test? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulk.java ########## @@ -0,0 +1,59 @@ +/* + * 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.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.jobmaster.SlotRequestId; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Represents a bulk of physical slot requests. + */ +public class PhysicalSlotRequestBulk { + + private final Map<SlotRequestId, ResourceProfile> pendingRequests; + + private final Map<SlotRequestId, AllocationID> fulfilledRequests = new HashMap<>(); + + PhysicalSlotRequestBulk(final Collection<PhysicalSlotRequest> physicalSlotRequests) { + this.pendingRequests = physicalSlotRequests.stream() + .collect(Collectors.toMap( + PhysicalSlotRequest::getSlotRequestId, + r -> r.getSlotProfile().getPhysicalSlotResourceProfile())); + } + + void markRequestFulfilled(final SlotRequestId slotRequestId, final AllocationID allocationID) { Review comment: ```suggestion void fulfillPendingSlotRequest(final SlotRequestId slotRequestId, final AllocationID allocationID) { ``` ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImpl.java ########## @@ -447,12 +452,40 @@ public void releaseSlot(@Nonnull SlotRequestId slotRequestId, @Nullable Throwabl componentMainThreadExecutor.assertRunningInMainThread(); + checkState(batchSlotTimeoutCheckEnabled, "batch slot timeout check is disabled unexpectedly."); + final PendingRequest pendingRequest = PendingRequest.createBatchRequest(slotRequestId, resourceProfile); return requestNewAllocatedSlotInternal(pendingRequest) .thenApply(Function.identity()); } + @Override + public CompletableFuture<PhysicalSlot> requestNewAllocatedSlotWithoutTimeout( + final SlotRequestId slotRequestId, + final ResourceProfile resourceProfile, + final boolean isBatchRequest) { + + componentMainThreadExecutor.assertRunningInMainThread(); + + // a slot pool can serve either this kind of request or requestNewAllocatedBatchSlot(...), never both + disableBatchSlotTimeoutCheck(); Review comment: ```suggestion batchSlotTimeoutCheckEnabled = false; ``` do we need a method for this? ---------------------------------------------------------------- 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]
