zentol commented on a change in pull request #13553: URL: https://github.com/apache/flink/pull/13553#discussion_r503168361
########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java ########## @@ -0,0 +1,1092 @@ +/* + * 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.resourcemanager.slotmanager; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple6; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.clusterframework.types.SlotID; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor; +import org.apache.flink.runtime.concurrent.ScheduledExecutor; +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.resourcemanager.ResourceManagerId; +import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec; +import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection; +import org.apache.flink.runtime.slots.ResourceRequirement; +import org.apache.flink.runtime.slots.ResourceRequirements; +import org.apache.flink.runtime.taskexecutor.SlotReport; +import org.apache.flink.runtime.taskexecutor.SlotStatus; +import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; +import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGateway; +import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; +import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException; +import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.FunctionUtils; + +import akka.pattern.AskTimeoutException; +import org.junit.Ignore; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.hamcrest.CoreMatchers.hasItem; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +/** + * Tests for the {@link DeclarativeSlotManager}. + */ +public class DeclarativeSlotManagerTest extends TestLogger { + + private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception"); + + private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder() + .setCpuCores(100.0) + .setTaskHeapMemoryMB(10000) + .setTaskOffHeapMemoryMB(10000) + .setNetworkMemoryMB(10000) + .setManagedMemoryMB(10000) + .build(); + + /** + * Tests that we can register task manager and their slots at the slot manager. + */ + @Test + public void testTaskManagerRegistration() throws Exception { + final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway(); + final ResourceID resourceId = ResourceID.generate(); + final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway); + + final SlotID slotId1 = new SlotID(resourceId, 0); + final SlotID slotId2 = new SlotID(resourceId, 1); + final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2))); + + final DefaultSlotTracker slotTracker = new DefaultSlotTracker(); + try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder() + .setSlotTracker(slotTracker) + .buildAndStartWithDirectExec()) { + + slotManager.registerTaskManager(taskManagerConnection, slotReport); + + assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2)); + + assertNotNull(slotTracker.getSlot(slotId1)); + assertNotNull(slotTracker.getSlot(slotId2)); + } + } + + /** + * Tests that un-registration of task managers will free and remove all registered slots. + */ + @Test + public void testTaskManagerUnregistration() throws Exception { + final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder() + .setRequestSlotFunction(tuple6 -> new CompletableFuture<>()) + .createTestingTaskExecutorGateway(); + final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway); + final ResourceID resourceId = taskManagerConnection.getResourceID(); + + final SlotID slotId1 = new SlotID(resourceId, 0); + final SlotID slotId2 = new SlotID(resourceId, 1); + final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2))); + + final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot(); + + final DefaultSlotTracker slotTracker = new DefaultSlotTracker(); + + try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder() + .setSlotTracker(slotTracker) + .buildAndStartWithDirectExec()) { + + slotManager.registerTaskManager(taskManagerConnection, slotReport); + + assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots()); + + DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1); + DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2); + + assertSame(SlotState.ALLOCATED, slot1.getState()); + assertSame(SlotState.FREE, slot2.getState()); + + slotManager.processResourceRequirements(resourceRequirements); + + assertSame(SlotState.PENDING, slot2.getState()); + + slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION); + + assertEquals(0, slotManager.getNumberRegisteredSlots()); + } + } + + /** + * Tests that a slot request with no free slots will trigger the resource allocation. + */ + @Test + public void testSlotRequestWithoutFreeSlots() throws Exception { + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); + + final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot(); + + CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>(); + ResourceActions resourceManagerActions = new TestingResourceActionsBuilder() + .setAllocateResourceConsumer(allocateResourceFuture::complete) + .build(); + + try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) { + + slotManager.processResourceRequirements(resourceRequirements); + + allocateResourceFuture.get(); + } + } + + /** + * Tests that resources continue to be considered missing if we cannot allocate more resources. + */ + @Test + public void testResourceDeclarationWithResourceAllocationFailure() throws Exception { + final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot(); + + ResourceActions resourceManagerActions = new TestingResourceActionsBuilder() + .setAllocateResourceFunction(value -> false) + .build(); + + final ResourceTracker resourceTracker = new DefaultResourceTracker(); + + try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder() + .setResourceTracker(resourceTracker) + .buildAndStartWithDirectExec(ResourceManagerId.generate(), resourceManagerActions)) { + + slotManager.processResourceRequirements(resourceRequirements); + + final JobID jobId = resourceRequirements.getJobId(); + assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1)); + } + } + + /** + * Tests that a slot request which can be fulfilled will trigger a slot allocation. + */ + @Test + public void testSlotRequestWithFreeSlot() throws Exception { + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); + final ResourceID resourceID = ResourceID.generate(); + final JobID jobId = new JobID(); + final SlotID slotId = new SlotID(resourceID, 0); + final String targetAddress = "localhost"; + final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337); + + final DefaultSlotTracker slotTracker = new DefaultSlotTracker(); + + try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder() + .setSlotTracker(slotTracker) + .buildAndStartWithDirectExec(resourceManagerId, new TestingResourceActionsBuilder().build())) { + + final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>(); + // accept an incoming slot request + final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder() + .setRequestSlotFunction(tuple6 -> { + requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5)); + return CompletableFuture.completedFuture(Acknowledge.get()); + }) + .createTestingTaskExecutorGateway(); + + final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway); + + final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile); + final SlotReport slotReport = new SlotReport(slotStatus); + + slotManager.registerTaskManager( + taskExecutorConnection, + slotReport); + + ResourceRequirements requirements = ResourceRequirements.create( + jobId, + targetAddress, + Collections.singleton(ResourceRequirement.create(resourceProfile, 1))); + slotManager.processResourceRequirements(requirements); + + assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId)))); + + DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId); + + assertEquals("The slot has not been allocated to the expected allocation id.", jobId, slot.getJobId()); + } + } + + /** + * Checks that reducing resource requirements while a slot allocation is in-progress results in the slot being freed + * one the allocation completes. + */ + @Test + @Ignore + public void testResourceRequirementReductionDuringAllocation() throws Exception { Review comment: No, that's why it I added `@Ignore` initially, but I never got around to deleting it. Once the requirements are reduced the SlotManager knows that the pending slots exceeds the requirements, but takes no further action. The TaskExecutor will offer the slot to the JM, which may or may not fail. In any case the TM will inform the SM about the resulting state, so we're good in regards to correctness. (And we'd introduce an odd edge-case where the SM actually frees a slot on the TM) ---------------------------------------------------------------- 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]
