tillrohrmann commented on a change in pull request #14465:
URL: https://github.com/apache/flink/pull/14465#discussion_r549766548



##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/TestingPhysicalSlotProvider.java
##########
@@ -69,71 +63,43 @@ private 
TestingPhysicalSlotProvider(SlotSharingExecutionSlotAllocatorTest.Physic
                this.requests = new HashMap<>();
                this.responses = new HashMap<>();
                this.cancellations = new HashMap<>();
-
-               this.availableSlotCount = availableSlotCount;
-               this.allocateLatch = new CountDownLatch(availableSlotCount);
        }
 
        @Override
        public CompletableFuture<PhysicalSlotRequest.Result> 
allocatePhysicalSlot(PhysicalSlotRequest physicalSlotRequest) {
                SlotRequestId slotRequestId = 
physicalSlotRequest.getSlotRequestId();
                requests.put(slotRequestId, physicalSlotRequest);
-               CompletableFuture<TestingPhysicalSlot> resultFuture = new 
CompletableFuture<>();
-               responses.put(slotRequestId, resultFuture);
-
-               switch (physicalSlotFutureCompletion) {
-                       case SUCCESS:
-                               if (availableSlotCount > 0) {
-                                       
completePhysicalSlotFutureFor(slotRequestId, new AllocationID());
-                               } else {
-                                       resultFuture.completeExceptionally(new 
NoResourceAvailableException());
-                               }
-                               break;
-                       case FAILURE:
-                               resultFuture.completeExceptionally(new 
FlinkException("Test failure."));
-                               break;
+               CompletableFuture<TestingPhysicalSlot> resultFuture;
+               try {
+                       resultFuture = physicalSlotCreator.apply(
+                               taskManagerLocation,
+                               taskManagerGateway,
+                               
physicalSlotRequest.getSlotProfile().getPhysicalSlotResourceProfile());
+               } catch (Throwable t) {
+                       resultFuture = FutureUtils.completedExceptionally(t);

Review comment:
       Given that `physicalSlotCreator` can return a `CompletableFuture`, why 
do we need to catch exceptions here? Couldn't the creator return an 
exceptionally completed future? If the creator only returns a 
`TestingPhysicalSlot`, then this would be needed.

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/TestingPhysicalSlotProvider.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import 
org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway;
+import 
org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
+import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.TriFunctionWithException;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * A {@link PhysicalSlotProvider} implementation that can be used in tests.
+ */
+public class TestingPhysicalSlotProvider implements PhysicalSlotProvider {
+
+       private final Map<SlotRequestId, PhysicalSlotRequest> requests;
+       private final Map<SlotRequestId, 
CompletableFuture<TestingPhysicalSlot>> responses;
+       private final Map<SlotRequestId, Throwable> cancellations;
+
+       private final TaskManagerLocation taskManagerLocation;
+       private final TaskManagerGateway taskManagerGateway;
+
+       private final TriFunctionWithException<TaskManagerLocation, 
TaskManagerGateway, ResourceProfile, CompletableFuture<TestingPhysicalSlot>, 
Throwable> physicalSlotCreator;
+
+       private 
TestingPhysicalSlotProvider(TriFunctionWithException<TaskManagerLocation, 
TaskManagerGateway, ResourceProfile, CompletableFuture<TestingPhysicalSlot>, 
Throwable> physicalSlotCreator, TaskManagerLocation taskManagerLocation, 
TaskManagerGateway taskManagerGateway) {
+               this.physicalSlotCreator = physicalSlotCreator;
+
+               this.taskManagerLocation = taskManagerLocation;
+               this.taskManagerGateway = taskManagerGateway;
+
+               this.requests = new HashMap<>();
+               this.responses = new HashMap<>();
+               this.cancellations = new HashMap<>();
+       }
+
+       @Override
+       public CompletableFuture<PhysicalSlotRequest.Result> 
allocatePhysicalSlot(PhysicalSlotRequest physicalSlotRequest) {
+               SlotRequestId slotRequestId = 
physicalSlotRequest.getSlotRequestId();
+               requests.put(slotRequestId, physicalSlotRequest);
+               CompletableFuture<TestingPhysicalSlot> resultFuture;
+               try {
+                       resultFuture = physicalSlotCreator.apply(
+                               taskManagerLocation,
+                               taskManagerGateway,
+                               
physicalSlotRequest.getSlotProfile().getPhysicalSlotResourceProfile());

Review comment:
       I am still a bit sceptical that `taskManagerLocation` and 
`taskManagerGateway` belong to the `TestingPhysicalSlotProvider`. I think they 
rather belong to the `physicalSlotCreator`. An indicator is that we now have 2 
ways of how to specify the `TaskManagerLocation`. 1) set a custom 
`physicalSlotCreator` which is given the location 2) set the location for the 
`TestingPhysicalSlotProvider`. I think this is an indicator that there is a bit 
of ambiguity in the separation of concerns.




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to