zentol commented on a change in pull request #14851:
URL: https://github.com/apache/flink/pull/14851#discussion_r570176223



##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/declarative/allocator/SlotSharingSlotAllocatorTest.java
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.declarative.allocator;
+
+import org.apache.flink.api.common.operators.ResourceSpec;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.runtime.jobmaster.slotpool.ResourceCounter;
+import org.apache.flink.runtime.scheduler.TestingPhysicalSlot;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.contains;
+import static org.junit.Assert.assertThat;
+
+/** Tests for the {@link SlotSharingSlotAllocator}. */
+public class SlotSharingSlotAllocatorTest extends TestLogger {
+
+    private static final FreeSlotFunction TEST_FREE_SLOT_FUNCTION = (a, c, t) 
-> {};
+    private static final ReserveSlotFunction TEST_RESERVE_SLOT_FUNCTION =
+            (allocationId, resourceProfile) ->
+                    TestingPhysicalSlot.builder()
+                            .withAllocationID(allocationId)
+                            .withResourceProfile(resourceProfile)
+                            .build();
+
+    private static final SlotSharingGroup slotSharingGroup1 = new 
SlotSharingGroup();
+    private static final SlotSharingGroup slotSharingGroup2 = new 
SlotSharingGroup();
+    private static final JobInformation.VertexInformation vertex1 =
+            new TestVertexInformation(new JobVertexID(), 4, slotSharingGroup1);
+    private static final JobInformation.VertexInformation vertex2 =
+            new TestVertexInformation(new JobVertexID(), 2, slotSharingGroup1);
+    private static final JobInformation.VertexInformation vertex3 =
+            new TestVertexInformation(new JobVertexID(), 3, slotSharingGroup2);
+
+    @Test
+    public void testCalculateRequiredSlots() {
+        final SlotSharingSlotAllocator slotAllocator =
+                new SlotSharingSlotAllocator(TEST_RESERVE_SLOT_FUNCTION, 
TEST_FREE_SLOT_FUNCTION);
+
+        final ResourceCounter resourceCounter =
+                slotAllocator.calculateRequiredSlots(Arrays.asList(vertex1, 
vertex2, vertex3));
+
+        assertThat(resourceCounter.getResources(), 
contains(ResourceProfile.UNKNOWN));
+        assertThat(
+                resourceCounter.getResourceCount(ResourceProfile.UNKNOWN),
+                is(
+                        Math.max(vertex1.getParallelism(), 
vertex2.getParallelism())
+                                + vertex3.getParallelism()));
+    }
+
+    @Test
+    public void testDetermineParallelismWithMinimumSlots() {
+        final SlotSharingSlotAllocator slotAllocator =
+                new SlotSharingSlotAllocator(TEST_RESERVE_SLOT_FUNCTION, 
TEST_FREE_SLOT_FUNCTION);
+
+        final JobInformation jobInformation =
+                new TestJobInformation(Arrays.asList(vertex1, vertex2, 
vertex3));
+
+        final VertexParallelism slotSharingAssignments =
+                slotAllocator.determineParallelism(jobInformation, 
getSlots(2)).get();
+
+        final Map<JobVertexID, Integer> maxParallelismForVertices =
+                slotSharingAssignments.getMaxParallelismForVertices();
+
+        assertThat(maxParallelismForVertices.get(vertex1.getJobVertexID()), 
is(1));
+        assertThat(maxParallelismForVertices.get(vertex2.getJobVertexID()), 
is(1));
+        assertThat(maxParallelismForVertices.get(vertex3.getJobVertexID()), 
is(1));
+    }
+
+    @Test
+    public void testDetermineParallelismWithManySlots() {
+        final SlotSharingSlotAllocator slotAllocator =
+                new SlotSharingSlotAllocator(TEST_RESERVE_SLOT_FUNCTION, 
TEST_FREE_SLOT_FUNCTION);
+
+        final JobInformation jobInformation =
+                new TestJobInformation(Arrays.asList(vertex1, vertex2, 
vertex3));
+
+        final VertexParallelism slotSharingAssignments =
+                slotAllocator.determineParallelism(jobInformation, 
getSlots(50)).get();
+
+        final Map<JobVertexID, Integer> maxParallelismForVertices =
+                slotSharingAssignments.getMaxParallelismForVertices();
+
+        assertThat(
+                maxParallelismForVertices.get(vertex1.getJobVertexID()),
+                is(vertex1.getParallelism()));
+        assertThat(
+                maxParallelismForVertices.get(vertex2.getJobVertexID()),
+                is(vertex2.getParallelism()));
+        assertThat(
+                maxParallelismForVertices.get(vertex3.getJobVertexID()),
+                is(vertex3.getParallelism()));
+    }
+
+    @Test
+    public void 
testDetermineParallelismUnsuccessfulWithLessSlotsThanSlotSharingGroups() {
+        final SlotSharingSlotAllocator slotAllocator =
+                new SlotSharingSlotAllocator(TEST_RESERVE_SLOT_FUNCTION, 
TEST_FREE_SLOT_FUNCTION);
+
+        final JobInformation jobInformation =
+                new TestJobInformation(Arrays.asList(vertex1, vertex2, 
vertex3));
+
+        final Optional<SlotSharingAssignments> slotSharingAssignments =
+                slotAllocator.determineParallelism(jobInformation, 
getSlots(1));
+
+        assertThat(slotSharingAssignments.isPresent(), is(false));
+    }
+
+    @Test
+    public void testReserveResources() {
+        final SlotSharingSlotAllocator slotAllocator =
+                new SlotSharingSlotAllocator(TEST_RESERVE_SLOT_FUNCTION, 
TEST_FREE_SLOT_FUNCTION);
+
+        final JobInformation jobInformation =
+                new TestJobInformation(Arrays.asList(vertex1, vertex2, 
vertex3));
+
+        final SlotSharingAssignments slotAssignments =
+                slotAllocator.determineParallelism(jobInformation, 
getSlots(50)).get();
+
+        final Map<ExecutionVertexID, LogicalSlot> assignedResources =
+                slotAllocator.reserveResources(slotAssignments);
+
+        final Map<ExecutionVertexID, SlotInfo> expectedAssignments = new 
HashMap<>();
+        for (SlotSharingSlotAllocator.ExecutionSlotSharingGroupAndSlot 
assignment :
+                slotAssignments.getAssignments()) {
+            for (ExecutionVertexID containedExecutionVertex :
+                    
assignment.getExecutionSlotSharingGroup().getContainedExecutionVertices()) {
+                expectedAssignments.put(containedExecutionVertex, 
assignment.getSlotInfo());
+            }
+        }
+
+        for (Map.Entry<ExecutionVertexID, SlotInfo> expectedAssignment :
+                expectedAssignments.entrySet()) {
+            final LogicalSlot assignedSlot = 
assignedResources.get(expectedAssignment.getKey());
+
+            final SlotInfo backingSlot = expectedAssignment.getValue();
+
+            assertThat(assignedSlot.getAllocationId(), 
is(backingSlot.getAllocationId()));
+        }
+    }
+
+    private static Collection<SlotInfo> getSlots(int count) {
+        final Collection<SlotInfo> slotInfo = new ArrayList<>();
+        for (int i = 0; i < count; i++) {
+            slotInfo.add(new TestSlotInfo());
+        }
+        return slotInfo;
+    }
+
+    private static class TestJobInformation implements JobInformation {
+
+        private final Map<JobVertexID, VertexInformation> vertexInformation;
+        private final Collection<SlotSharingGroup> slotSharingGroups;
+
+        private TestJobInformation(Collection<VertexInformation> 
vertexInformation) {
+            this.vertexInformation =
+                    vertexInformation.stream()
+                            .collect(
+                                    Collectors.toMap(
+                                            VertexInformation::getJobVertexID,
+                                            Function.identity()));
+            this.slotSharingGroups =
+                    vertexInformation.stream()
+                            .map(VertexInformation::getSlotSharingGroup)
+                            .collect(
+                                    Collectors.toMap(
+                                            
SlotSharingGroup::getSlotSharingGroupId,
+                                            Function.identity(),
+                                            (slotSharingGroup1, 
slotSharingGroup2) ->
+                                                    slotSharingGroup1))

Review comment:
       why did I do this in such a complicated way...




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


Reply via email to