GJL commented on a change in pull request #8486: [FLINK-12372] [runtime] implement ExecutionSlotAllocator URL: https://github.com/apache/flink/pull/8486#discussion_r290331180
########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocatorTest.java ########## @@ -0,0 +1,355 @@ +/* + * 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.scheduler; + +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.executiongraph.TestingSlotProvider; +import org.apache.flink.runtime.instance.SlotSharingGroupId; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.runtime.jobgraph.DistributionPattern; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.jobmaster.SlotRequestId; +import org.apache.flink.runtime.jobmaster.TestingLogicalSlot; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.util.TestLogger; + +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.CompletableFuture; + +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.hasSize; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +/** + * Tests for {@link DefaultExecutionSlotAllocator}. + */ +public class DefaultExecutionSlotAllocatorTest extends TestLogger { + + private DefaultExecutionSlotAllocator executionSlotAllocator; + + private TestingSlotProvider slotProvider; + + private TestingInputsLocationsRetriever testingInputsLocationsRetriever; + + private List<ExecutionVertexSchedulingRequirements> schedulingRequirements; + + private Queue<CompletableFuture<LogicalSlot>> slotFutures; + + private List<SlotRequestId> receivedSlotRequestIds; + + private List<SlotRequestId> cancelledSlotRequestIds; + + private List<ExecutionVertexID> executionVertexIds; + + private int executionVerticesNum; + + @Before + public void setUp() throws Exception { + testingInputsLocationsRetriever = createSimpleInputsLocationsRetriever(); + + executionVerticesNum = testingInputsLocationsRetriever.getTotalNumberOfVertices(); + receivedSlotRequestIds = new ArrayList<>(executionVerticesNum); + cancelledSlotRequestIds = new ArrayList<>(executionVerticesNum); + slotFutures = new ArrayDeque<>(executionVerticesNum); + + slotProvider = new TestingSlotProvider(slotRequestId -> { + receivedSlotRequestIds.add(slotRequestId); + return slotFutures.poll(); + }); + slotProvider.setSlotCanceller(slotRequestId -> cancelledSlotRequestIds.add(slotRequestId)); + + executionSlotAllocator = new DefaultExecutionSlotAllocator( + slotProvider, + testingInputsLocationsRetriever, + Time.seconds(10), + true); + + schedulingRequirements = new ArrayList<>(executionVerticesNum); + executionVertexIds = new ArrayList<>(executionVerticesNum); + + for (ExecutionVertexID executionVertexId : testingInputsLocationsRetriever.getAllExecutionVertices()) { + schedulingRequirements.add(new ExecutionVertexSchedulingRequirements( + executionVertexId, + null, + ResourceProfile.UNKNOWN, + new SlotSharingGroupId(), + null, + null + )); + executionVertexIds.add(executionVertexId); + slotFutures.add(new CompletableFuture<>()); + } + } + + /** + * Tests that it will allocate slots from slot provider and remove the slot assignments when request are fulfilled. + */ + @Test + public void testAllocateSlotsFor() { + List<CompletableFuture<LogicalSlot>> backupSlotFutures = new ArrayList<>(slotFutures); + + executionSlotAllocator.allocateSlotsFor(schedulingRequirements); + + assertThat(receivedSlotRequestIds, hasSize(executionVerticesNum / 2)); + assertThat(executionSlotAllocator.getPendingSlotAssignments().keySet(), containsInAnyOrder(executionVertexIds.toArray())); + + completeProducerSlotRequest(backupSlotFutures); + + assertThat(receivedSlotRequestIds, hasSize(executionVerticesNum)); + } + + /** + * Tests that when cancelling a slot request, the request to slot provider should also be cancelled. + */ + @Test + public void testCancel() { + List<CompletableFuture<LogicalSlot>> backupSlotFutures = new ArrayList<>(slotFutures); + + executionSlotAllocator.allocateSlotsFor(schedulingRequirements); + + // cancel a non-existing execution vertex + ExecutionVertexID inValidExecutionVertexId = new ExecutionVertexID(new JobVertexID(), 0); + executionSlotAllocator.cancel(inValidExecutionVertexId); + assertThat(cancelledSlotRequestIds, hasSize(0)); + + assertThat(receivedSlotRequestIds, hasSize(executionVerticesNum / 2)); + + completeProducerSlotRequest(backupSlotFutures); + + for (ExecutionVertexID executionVertexId : executionVertexIds) { + executionSlotAllocator.cancel(executionVertexId); + } + + // only the consumers slot request will be cancelled + List<SlotRequestId> expectCancelledSlotRequestIds = + receivedSlotRequestIds.subList(executionVerticesNum / 2, executionVerticesNum); + assertThat(cancelledSlotRequestIds, containsInAnyOrder(expectCancelledSlotRequestIds.toArray())); + } + + /** + * Tests that all unfulfilled slot requests will be cancelled when stopped. + */ + @Test + public void testStop() { + executionSlotAllocator.allocateSlotsFor(schedulingRequirements); + + // only the producers will ask for slots as the preferred location futures of consumers are not fulfilled. + assertThat(executionSlotAllocator.getPendingSlotAssignments().keySet(), hasSize(executionVerticesNum)); + assertThat(receivedSlotRequestIds, hasSize(executionVerticesNum / 2)); + + executionSlotAllocator.stop().getNow(null); + + assertThat(cancelledSlotRequestIds, hasSize(executionVerticesNum / 2)); + assertThat(cancelledSlotRequestIds, containsInAnyOrder(receivedSlotRequestIds.toArray())); + assertThat(executionSlotAllocator.getPendingSlotAssignments().keySet(), hasSize(0)); + } + + /** + * Tests that all prior allocation ids are computed by union all previous allocation ids in scheduling requirements. + */ + @Test + public void testComputeAllPriorAllocationIds() { + List<AllocationID> expectAllocationIds = Arrays.asList(new AllocationID(), new AllocationID()); + List<ExecutionVertexSchedulingRequirements> testSchedulingRequirements = Arrays.asList( + new ExecutionVertexSchedulingRequirements( + executionVertexIds.get(0), + expectAllocationIds.get(0), + ResourceProfile.UNKNOWN, + null, + null, + null), + new ExecutionVertexSchedulingRequirements( + executionVertexIds.get(1), + expectAllocationIds.get(0), + ResourceProfile.UNKNOWN, + null, + null, + null), + new ExecutionVertexSchedulingRequirements( + executionVertexIds.get(2), + expectAllocationIds.get(1), + ResourceProfile.UNKNOWN, + null, + null, + null), + new ExecutionVertexSchedulingRequirements( + executionVertexIds.get(3), + null, + ResourceProfile.UNKNOWN, + null, + null, + null) + ); + + Set<AllocationID> allPriorAllocationIds = executionSlotAllocator.computeAllPriorAllocationIds(testSchedulingRequirements); + assertThat(allPriorAllocationIds, containsInAnyOrder(expectAllocationIds.toArray())); + } + + /** + * Tests the calculation of preferred locations based on inputs for an execution. + */ + @Test + public void testGetPreferredLocationsBasedOnInputs() throws Exception { + JobVertex producer1 = new JobVertex("producer1"); + producer1.setInvokableClass(NoOpInvokable.class); Review comment: Setting the invokable class is not needed and distracts from the actual test logic. ---------------------------------------------------------------- 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] With regards, Apache Git Services
