GJL commented on a change in pull request #8486: [FLINK-12372] [runtime] implement ExecutionSlotAllocator URL: https://github.com/apache/flink/pull/8486#discussion_r290340620
########## 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() { Review comment: Here are some comments/suggestions: - `TestingInputsLocationsRetriever` has a dependency to `JobVertex` – I don't think this dependency is justified. - Job graph always has a parallelism of 3. However, a parallelism higher than 1 is not necessary relevant for all tests. - It is implicitly encoded in `createSimpleInputsLocationsRetriever()` that the first half of the execution vertices are producers (the other half are consumers). This makes the tests and the assertions hard to read because I have to first read `createSimpleInputsLocationsRetriever()` in `setUp()` to understand what the test asserts. - The tests are not asserting the contract of `ExecutionSlotAllocator`: - Tests should assert on the public methods of the interface, i.e., we should assert against the return type of `allocateSlotsFor` (`Collection<SlotExecutionVertexAssignment>`) - `@VisibleForTesting` is a code smell - test name should describe what is being tested, e.g., the expected behavior Find below a draft of how the tests could look like. Feel free to adapt. ``` @Test public void producersAreAssignedToSlotsBeforeConsumers() { final ExecutionVertexID producerId = new ExecutionVertexID(new JobVertexID(), 0); final ExecutionVertexID consumerId = new ExecutionVertexID(new JobVertexID(), 0); final TestingInputLocationRetriever inputLocationRetriever = new TestingInputLocationRetriever.Builder() .connectConsumerToProducer(consumerId, producerId) .build(); final DefaultExecutionSlotAllocator executionSlotAllocator = createExecutionSlotAllocator(inputLocationRetriever); inputLocationRetriever.markScheduled(producerId); inputLocationRetriever.markScheduled(consumerId); final List<ExecutionVertexSchedulingRequirements> schedulingRequirements = createSchedulingRequirements(producerId, consumerId); final Collection<SlotExecutionVertexAssignment> slotExecutionVertexAssignments = executionSlotAllocator.allocateSlotsFor(schedulingRequirements); final SlotExecutionVertexAssignment producerSlotAssignment = findSlotAssignmentByExecutionVertexId(producerId, slotExecutionVertexAssignments); final SlotExecutionVertexAssignment consumerSlotAssignment = findSlotAssignmentByExecutionVertexId(consumerId, slotExecutionVertexAssignments); assertTrue(producerSlotAssignment.getLogicalSlotFuture().isDone()); assertFalse(consumerSlotAssignment.getLogicalSlotFuture().isDone()); } @Test public void consumersAreAssignedToSlotsAfterProducers() { final ExecutionVertexID producerId = new ExecutionVertexID(new JobVertexID(), 0); final ExecutionVertexID consumerId = new ExecutionVertexID(new JobVertexID(), 0); final TestingInputLocationRetriever inputLocationRetriever = new TestingInputLocationRetriever.Builder() .connectConsumerToProducer(consumerId, producerId) .build(); final DefaultExecutionSlotAllocator executionSlotAllocator = createExecutionSlotAllocator(inputLocationRetriever); inputLocationRetriever.markScheduled(producerId); inputLocationRetriever.markScheduled(consumerId); final List<ExecutionVertexSchedulingRequirements> schedulingRequirements = createSchedulingRequirements(producerId, consumerId); final Collection<SlotExecutionVertexAssignment> slotExecutionVertexAssignments = executionSlotAllocator.allocateSlotsFor(schedulingRequirements); inputLocationRetriever.assignTaskManagerLocation(producerId); final SlotExecutionVertexAssignment consumerSlotAssignment = findSlotAssignmentByExecutionVertexId(consumerId, slotExecutionVertexAssignments); assertTrue(consumerSlotAssignment.getLogicalSlotFuture().isDone()); } ``` ``` ``` class TestingInputLocationRetriever implements InputsLocationsRetriever { private final Map<ExecutionVertexID, List<ExecutionVertexID>> producersByVertex; private final Map<ExecutionVertexID, CompletableFuture<TaskManagerLocation>> taskManagerLocationsByVertex = new HashMap<>(); TestingInputLocationRetriever(final Map<ExecutionVertexID, List<ExecutionVertexID>> producersByVertex) { this.producersByVertex = new HashMap<>(producersByVertex); } @Override public Collection<Collection<ExecutionVertexID>> getConsumedResultPartitionsProducers(final ExecutionVertexID executionVertexId) { final Map<JobVertexID, List<ExecutionVertexID>> executionVerticesByJobVertex = producersByVertex.getOrDefault(executionVertexId, Collections.emptyList()) .stream() .collect(Collectors.groupingBy(ExecutionVertexID::getJobVertexId)); return new ArrayList<>(executionVerticesByJobVertex.values()); } @Override public Optional<CompletableFuture<TaskManagerLocation>> getTaskManagerLocation(final ExecutionVertexID executionVertexId) { return Optional.ofNullable(taskManagerLocationsByVertex.get(executionVertexId)); } public void markScheduled(final ExecutionVertexID executionVertexId) { taskManagerLocationsByVertex.put(executionVertexId, new CompletableFuture<>()); } public void assignTaskManagerLocation(final ExecutionVertexID executionVertexId) { taskManagerLocationsByVertex.compute(executionVertexId, (key, future) -> { if (future == null) { return CompletableFuture.completedFuture(new LocalTaskManagerLocation()); } future.complete(new LocalTaskManagerLocation()); return future; }); } static class Builder { private final Map<ExecutionVertexID, List<ExecutionVertexID>> consumerToProducers = new HashMap<>(); public Builder connectConsumerToProducer(final ExecutionVertexID consumer, final ExecutionVertexID producer) { consumerToProducers.compute(consumer, (key, producers) -> { if (producers == null) { producers = new ArrayList<>(); } producers.add(producer); return producers; }); return this; } public TestingInputLocationRetriever build() { return new TestingInputLocationRetriever(consumerToProducers); } } } ``` ---------------------------------------------------------------- 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
