azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r481295508



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SharingPhysicalSlotRequestBulk.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.jobmaster.slotpool.PhysicalSlotRequestBulk;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.stream.Stream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+class SharingPhysicalSlotRequestBulk implements PhysicalSlotRequestBulk {
+       private final Map<ExecutionSlotSharingGroup, List<ExecutionVertexID>> 
executions;
+
+       private final Map<ExecutionSlotSharingGroup, ResourceProfile> 
pendingRequests;
+
+       private final Map<ExecutionSlotSharingGroup, AllocationID> 
fulfilledRequests;
+
+       private final BiConsumer<ExecutionVertexID, Throwable> 
logicalSlotRequestCanceller;
+
+       SharingPhysicalSlotRequestBulk(
+                       Map<ExecutionSlotSharingGroup, List<ExecutionVertexID>> 
executions,
+                       Map<ExecutionSlotSharingGroup, ResourceProfile> 
pendingRequests,
+                       Map<ExecutionSlotSharingGroup, AllocationID> 
fulfilledRequests,
+                       BiConsumer<ExecutionVertexID, Throwable> 
logicalSlotRequestCanceller) {
+               this.executions = checkNotNull(executions);
+               this.pendingRequests = checkNotNull(pendingRequests);
+               this.fulfilledRequests = checkNotNull(fulfilledRequests);
+               this.logicalSlotRequestCanceller = 
checkNotNull(logicalSlotRequestCanceller);
+       }
+
+       @Override
+       public Collection<ResourceProfile> getPendingRequests() {
+               return pendingRequests.values();
+       }
+
+       @Override
+       public Set<AllocationID> getAllocationIdsOfFulfilledRequests() {
+               return new HashSet<>(fulfilledRequests.values());
+       }
+
+       @Override
+       public void cancel(Throwable cause) {
+               // pending requests must be canceled first otherwise they might 
be fulfilled by
+               // allocated slots released from this bulk
+               Stream
+                       .concat(
+                               pendingRequests.keySet().stream(),
+                               fulfilledRequests.keySet().stream())
+                       .flatMap(group -> executions.get(group).stream())
+                       .forEach(id -> logicalSlotRequestCanceller.accept(id, 
cause));

Review comment:
       This way we will have two duplicated loops for `pendingRequests` and 
`fulfilledRequests` or an intermediate container.
   the final `forEach` is supposed to produce side effects.
   I think it is a matter of taste here but I can change it.




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