You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/08/17 20:07:21 UTC

[GitHub] [flink] azagrebin opened a new pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

azagrebin opened a new pull request #13181:
URL: https://github.com/apache/flink/pull/13181


   Track fulfil-ability of required physical slots for all SharedSlot(s) (no matter whether they are created at this bulk or not) with timeout. This ensures we will not wait indefinitely if the required slots for this bulk cannot be fully fulfilled at the same time.
   - Create a LogicalSlotRequestBulk to track all physical requests and logical slot requests (logical slot requests only which belong to the bulk)
   - Mark physical slot request fulfilled in LogicalSlotRequestBulk, once its future is done
   - If any physical slot request fails then clear the LogicalSlotRequestBulk to stop the fulfil-ability check
   - Schedule a fulfil-ability check in LogicalSlotRequestBulkChecker for the LogicalSlotRequestBulk
   - In case of timeout cancel/fail the logical slot futures of the bulk in SharedSlot(s)


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



[GitHub] [flink] tillrohrmann commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r481794646



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkCheckerImpl.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.jobmaster.slotpool;
+
+import org.apache.flink.annotation.VisibleForTesting;
+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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.util.clock.Clock;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+class PhysicalSlotRequestBulkCheckerImpl implements PhysicalSlotRequestBulkChecker {
+
+	private ComponentMainThreadExecutor componentMainThreadExecutor;
+
+	private final Supplier<Set<SlotInfo>> slotsRetriever;
+
+	private final Clock clock;
+
+	PhysicalSlotRequestBulkCheckerImpl(final Supplier<Set<SlotInfo>> slotsRetriever, final Clock clock) {
+		this.slotsRetriever = checkNotNull(slotsRetriever);
+		this.clock = checkNotNull(clock);
+	}
+
+	void start(final ComponentMainThreadExecutor mainThreadExecutor) {
+		this.componentMainThreadExecutor = mainThreadExecutor;
+	}
+
+	@Override
+	public void schedulePendingRequestBulkTimeoutCheck(PhysicalSlotRequestBulk bulk, Time timeout) {
+		PhysicalSlotRequestBulkWithTimestamp bulkWithTimestamp = new PhysicalSlotRequestBulkWithTimestamp(bulk);
+		bulkWithTimestamp.markUnfulfillable(clock.relativeTimeMillis());
+		schedulePendingRequestBulkTimeoutCheck(bulkWithTimestamp, timeout);
+	}
+
+	private void schedulePendingRequestBulkTimeoutCheck(PhysicalSlotRequestBulkWithTimestamp bulk, Time timeout) {
+		componentMainThreadExecutor.schedule(() -> {
+			TimeoutCheckResult result = checkPhysicalSlotRequestBulkTimeout(bulk, timeout);
+
+			switch (result) {
+				case PENDING:
+					//re-schedule the timeout check
+					schedulePendingRequestBulkTimeoutCheck(bulk, timeout);
+					break;
+				case TIMEOUT:
+					bulk.cancel(new TimeoutException("Slot request bulk is not fulfillable!"));
+					break;
+				case FULFILLED:
+				default:
+					// no action to take
+					break;
+			}
+		}, timeout.getSize(), timeout.getUnit());
+	}
+
+	/**
+	 * Check the slot request bulk and timeout its requests if it has been unfulfillable for too long.
+	 * @param slotRequestBulk bulk of slot requests
+	 * @param slotRequestTimeout indicates how long a pending request can be unfulfillable
+	 * @return result of the check, indicating the bulk is fulfilled, still pending, or timed out
+	 */
+	@VisibleForTesting
+	TimeoutCheckResult checkPhysicalSlotRequestBulkTimeout(
+			final PhysicalSlotRequestBulkWithTimestamp slotRequestBulk,
+			final Time slotRequestTimeout) {
+
+		if (slotRequestBulk.getPendingRequests().isEmpty()) {
+			return TimeoutCheckResult.FULFILLED;
+		}
+
+		final boolean fulfillable = isSlotRequestBulkFulfillable(slotRequestBulk, slotsRetriever);
+		if (fulfillable) {
+			slotRequestBulk.markFulfillable();
+		} else {
+			final long currentTimestamp = clock.relativeTimeMillis();
+
+			slotRequestBulk.markUnfulfillable(currentTimestamp);
+
+			final long unfulfillableSince = slotRequestBulk.getUnfulfillableSince();
+			if (unfulfillableSince + slotRequestTimeout.toMilliseconds() <= currentTimestamp) {
+				return TimeoutCheckResult.TIMEOUT;
+			}
+		}
+
+		return TimeoutCheckResult.PENDING;
+	}
+
+	/**
+	 * Returns whether the given bulk of slot requests are possible to be fulfilled at the same time
+	 * with all the reusable slots in the slot pool. A reusable slot means the slot is available or
+	 * will not be occupied indefinitely.
+	 *
+	 * @param slotRequestBulk bulk of slot requests to check
+	 * @param slotsRetriever supplies slots to be used for the fulfill-ability check
+	 * @return true if the slot requests are possible to be fulfilled, otherwise false
+	 */
+	@VisibleForTesting
+	static boolean isSlotRequestBulkFulfillable(
+			final PhysicalSlotRequestBulk slotRequestBulk,
+			final Supplier<Set<SlotInfo>> slotsRetriever) {
+
+		final Set<AllocationID> assignedSlots = slotRequestBulk.getAllocationIdsOfFulfilledRequests();
+		final Set<SlotInfo> reusableSlots = getReusableSlots(slotsRetriever, assignedSlots);
+		return areRequestsFulfillableWithSlots(slotRequestBulk.getPendingRequests(), reusableSlots);
+	}
+
+	private static Set<SlotInfo> getReusableSlots(
+			final Supplier<Set<SlotInfo>> slotsRetriever,
+			final Set<AllocationID> slotsToExclude) {
+
+		return slotsRetriever.get().stream()
+			.filter(slotInfo -> !slotInfo.willBeOccupiedIndefinitely())
+			.filter(slotInfo -> !slotsToExclude.contains(slotInfo.getAllocationId()))
+			.collect(Collectors.toSet());
+	}
+
+	private static boolean areRequestsFulfillableWithSlots(
+			final Collection<ResourceProfile> requestResourceProfiles,
+			final Set<SlotInfo> slots) {
+
+		final Set<SlotInfo> remainingSlots = new HashSet<>(slots);
+		for (ResourceProfile requestResourceProfile : requestResourceProfiles) {
+			final Optional<SlotInfo> matchedSlot = findMatchingSlotForRequest(requestResourceProfile, remainingSlots);
+			if (matchedSlot.isPresent()) {
+				remainingSlots.remove(matchedSlot.get());
+			} else {
+				return false;
+			}
+		}
+		return true;
+	}
+
+	private static Optional<SlotInfo> findMatchingSlotForRequest(
+			final ResourceProfile requestResourceProfile,
+			final Collection<SlotInfo> slots) {
+
+		return slots.stream().filter(slot -> slot.getResourceProfile().isMatching(requestResourceProfile)).findFirst();
+	}
+
+	static PhysicalSlotRequestBulkCheckerImpl fromSlotPool(final SlotPool slotPool, final Clock clock) {
+		return new PhysicalSlotRequestBulkCheckerImpl(() -> getAllSlotInfos(slotPool), clock);
+	}
+
+	private static Set<SlotInfo> getAllSlotInfos(SlotPool slotPool) {
+		return Stream
+			.concat(
+				slotPool.getAvailableSlotsInformation().stream(),
+				slotPool.getAllocatedSlotsInformation().stream())
+			.collect(Collectors.toSet());

Review comment:
       This comment is wrong. `SlotPool.getAlocatedSlotsInformation` does not return all slot infos.




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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r481277688



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkChecker.java
##########
@@ -18,131 +18,24 @@
 
 package org.apache.flink.runtime.jobmaster.slotpool;
 
-import org.apache.flink.annotation.VisibleForTesting;
 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.jobmaster.SlotInfo;
-import org.apache.flink.util.clock.Clock;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Optional;
-import java.util.Set;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * This class helps to check the status of physical slot request bulks.
+ * This class tracks a fulfil-ability timeout of a bulk of physical slot requests.
+ *
+ * <p>The check stops when all physical slot requests of the bulk are fulfilled by available or newly allocated slots.
+ * The bulk is fulfillable if all its physical slot requests can be fulfilled either by available or
+ * newly allocated slots or slots which currently used by other job subtasks.
+ * The bulk gets canceled if the timeout occurs and the bulk is not fulfillable.
+ * The timeout does not tick while the bulk is fulfillable but not fulfilled yet.

Review comment:
       `trigger` also fits
   I meant that `the timeout timer is not running` which is a bit more precise to say imo




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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 347df14dd8621cf7922f723aa5faf96a4f4fc1c9 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671) 
   * 7e07e70b62850e70644e35b18c9168426371862c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r484706282



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SharedSlotTest.java
##########
@@ -234,6 +236,45 @@ public void testReleaseIfPhysicalSlotIsAllocated() {
 		assertThat(released.isDone(), is(true));
 	}
 
+	@Test
+	public void tesDuplicatedReturnLogicalSlotFails() {
+		CompletableFuture<PhysicalSlot> slotContextFuture = CompletableFuture
+			.completedFuture(new TestingPhysicalSlot(RP, new AllocationID()));
+		AtomicInteger released = new AtomicInteger(0);
+		SharedSlot sharedSlot = SharedSlotBuilder
+			.newBuilder()
+			.withSlotContextFuture(slotContextFuture)
+			.withReleaseCallback(g -> released.incrementAndGet())

Review comment:
       I think this case is already covered by `testReleaseEmptyDoesNotCallAllocatorReleaseBack` where `returnLogicalSlot` calls the callback and then next `sharedSlot.release` checks that the callback is not called any more. It is not exactly the same but the infinite recursion cannot happen and the recursive setup will be a bit complicated.




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



[GitHub] [flink] tillrohrmann commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r483678060



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkCheckerImpl.java
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.jobmaster.slotpool;
+
+import org.apache.flink.annotation.VisibleForTesting;
+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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.util.clock.Clock;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Default implementation of {@link PhysicalSlotRequestBulkChecker}.
+ */
+public class PhysicalSlotRequestBulkCheckerImpl implements PhysicalSlotRequestBulkChecker {
+
+	private ComponentMainThreadExecutor componentMainThreadExecutor;

Review comment:
       I'd initialize this field with a `DummyComponentMainThreadExecutor` to ensure that `start` is called before any other action is called on it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SharedSlot.java
##########
@@ -38,6 +38,26 @@
 import java.util.function.Consumer;
 import java.util.function.Function;
 
+/**
+ * Shared slot implementation for the {@link SlotSharingExecutionSlotAllocator}.
+ *
+ * <p>The shared slots are owned and tracked by {@link SlotSharingExecutionSlotAllocator}.
+ * The shared slot represents a collection of {@link SingleLogicalSlot} requests which share one physical slot.
+ * The shared slot is created by the {@link SlotSharingExecutionSlotAllocator} from the physical slot request.
+ * Afterwards, {@link SlotSharingExecutionSlotAllocator} requests logical slots from the underlying physical slot
+ * for execution which share it.

Review comment:
       ```suggestion
    * for {@link Execution executions} which share it.
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkCheckerImpl.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.jobmaster.slotpool;
+
+import org.apache.flink.annotation.VisibleForTesting;
+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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.util.clock.Clock;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+class PhysicalSlotRequestBulkCheckerImpl implements PhysicalSlotRequestBulkChecker {
+
+	private ComponentMainThreadExecutor componentMainThreadExecutor;
+
+	private final Supplier<Set<SlotInfo>> slotsRetriever;
+
+	private final Clock clock;
+
+	PhysicalSlotRequestBulkCheckerImpl(final Supplier<Set<SlotInfo>> slotsRetriever, final Clock clock) {
+		this.slotsRetriever = checkNotNull(slotsRetriever);
+		this.clock = checkNotNull(clock);
+	}
+
+	void start(final ComponentMainThreadExecutor mainThreadExecutor) {
+		this.componentMainThreadExecutor = mainThreadExecutor;
+	}
+
+	@Override
+	public void schedulePendingRequestBulkTimeoutCheck(PhysicalSlotRequestBulk bulk, Time timeout) {
+		PhysicalSlotRequestBulkWithTimestamp bulkWithTimestamp = new PhysicalSlotRequestBulkWithTimestamp(bulk);
+		bulkWithTimestamp.markUnfulfillable(clock.relativeTimeMillis());
+		schedulePendingRequestBulkTimeoutCheck(bulkWithTimestamp, timeout);
+	}
+
+	private void schedulePendingRequestBulkTimeoutCheck(PhysicalSlotRequestBulkWithTimestamp bulk, Time timeout) {
+		componentMainThreadExecutor.schedule(() -> {
+			TimeoutCheckResult result = checkPhysicalSlotRequestBulkTimeout(bulk, timeout);
+
+			switch (result) {
+				case PENDING:
+					//re-schedule the timeout check
+					schedulePendingRequestBulkTimeoutCheck(bulk, timeout);
+					break;
+				case TIMEOUT:
+					bulk.cancel(new TimeoutException("Slot request bulk is not fulfillable!"));
+					break;
+				case FULFILLED:
+				default:
+					// no action to take
+					break;
+			}
+		}, timeout.getSize(), timeout.getUnit());
+	}
+
+	/**
+	 * Check the slot request bulk and timeout its requests if it has been unfulfillable for too long.
+	 * @param slotRequestBulk bulk of slot requests
+	 * @param slotRequestTimeout indicates how long a pending request can be unfulfillable
+	 * @return result of the check, indicating the bulk is fulfilled, still pending, or timed out
+	 */
+	@VisibleForTesting
+	TimeoutCheckResult checkPhysicalSlotRequestBulkTimeout(
+			final PhysicalSlotRequestBulkWithTimestamp slotRequestBulk,
+			final Time slotRequestTimeout) {
+
+		if (slotRequestBulk.getPendingRequests().isEmpty()) {
+			return TimeoutCheckResult.FULFILLED;
+		}
+
+		final boolean fulfillable = isSlotRequestBulkFulfillable(slotRequestBulk, slotsRetriever);
+		if (fulfillable) {
+			slotRequestBulk.markFulfillable();
+		} else {
+			final long currentTimestamp = clock.relativeTimeMillis();
+
+			slotRequestBulk.markUnfulfillable(currentTimestamp);
+
+			final long unfulfillableSince = slotRequestBulk.getUnfulfillableSince();
+			if (unfulfillableSince + slotRequestTimeout.toMilliseconds() <= currentTimestamp) {
+				return TimeoutCheckResult.TIMEOUT;
+			}
+		}
+
+		return TimeoutCheckResult.PENDING;
+	}
+
+	/**
+	 * Returns whether the given bulk of slot requests are possible to be fulfilled at the same time
+	 * with all the reusable slots in the slot pool. A reusable slot means the slot is available or
+	 * will not be occupied indefinitely.
+	 *
+	 * @param slotRequestBulk bulk of slot requests to check
+	 * @param slotsRetriever supplies slots to be used for the fulfill-ability check
+	 * @return true if the slot requests are possible to be fulfilled, otherwise false
+	 */
+	@VisibleForTesting
+	static boolean isSlotRequestBulkFulfillable(
+			final PhysicalSlotRequestBulk slotRequestBulk,
+			final Supplier<Set<SlotInfo>> slotsRetriever) {
+
+		final Set<AllocationID> assignedSlots = slotRequestBulk.getAllocationIdsOfFulfilledRequests();
+		final Set<SlotInfo> reusableSlots = getReusableSlots(slotsRetriever, assignedSlots);
+		return areRequestsFulfillableWithSlots(slotRequestBulk.getPendingRequests(), reusableSlots);
+	}
+
+	private static Set<SlotInfo> getReusableSlots(
+			final Supplier<Set<SlotInfo>> slotsRetriever,
+			final Set<AllocationID> slotsToExclude) {
+
+		return slotsRetriever.get().stream()
+			.filter(slotInfo -> !slotInfo.willBeOccupiedIndefinitely())
+			.filter(slotInfo -> !slotsToExclude.contains(slotInfo.getAllocationId()))
+			.collect(Collectors.toSet());
+	}
+
+	private static boolean areRequestsFulfillableWithSlots(
+			final Collection<ResourceProfile> requestResourceProfiles,
+			final Set<SlotInfo> slots) {
+
+		final Set<SlotInfo> remainingSlots = new HashSet<>(slots);
+		for (ResourceProfile requestResourceProfile : requestResourceProfiles) {
+			final Optional<SlotInfo> matchedSlot = findMatchingSlotForRequest(requestResourceProfile, remainingSlots);
+			if (matchedSlot.isPresent()) {
+				remainingSlots.remove(matchedSlot.get());
+			} else {
+				return false;
+			}
+		}

Review comment:
       Maybe add a comment stating the complexity.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SharedSlotTest.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.executiongraph.utils.SimpleAckingTaskManagerGateway;
+import org.apache.flink.runtime.jobmanager.scheduler.Locality;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.slotpool.DummyPayload;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.scheduler.SharedSlotTestingUtils.TestingPhysicalSlot;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation;
+
+import org.junit.Test;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.Consumer;
+
+import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createRandomExecutionVertexId;
+import static org.apache.flink.runtime.scheduler.SharedSlotTestingUtils.createExecutionSlotSharingGroup;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Test suite for {@link SharedSlot}.
+ */
+public class SharedSlotTest {

Review comment:
       ```suggestion
   public class SharedSlotTest extends TestLogger {
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -138,27 +133,38 @@
 
 	@Override
 	public void cancel(ExecutionVertexID executionVertexId) {
+		cancelLogicalSlotRequest(executionVertexId, null);
+	}
+
+	private void cancelLogicalSlotRequest(ExecutionVertexID executionVertexId, Throwable cause) {
 		ExecutionSlotSharingGroup executionSlotSharingGroup =
 			slotSharingStrategy.getExecutionSlotSharingGroup(executionVertexId);
 		checkNotNull(
 			executionSlotSharingGroup,
 			"There is no ExecutionSlotSharingGroup for ExecutionVertexID " + executionVertexId);
 		SharedSlot slot = sharedSlots.get(executionSlotSharingGroup);
 		if (slot != null) {
-			slot.cancelLogicalSlotRequest(executionVertexId, null);
+			slot.cancelLogicalSlotRequest(executionVertexId, cause);
 		} else {
-			LOG.debug("There is no slot for ExecutionSlotSharingGroup of ExecutionVertexID {}", executionVertexId);
+			LOG.debug("There is no SharedSlot for ExecutionSlotSharingGroup of ExecutionVertexID {}", executionVertexId);
 		}
 	}
 
-	private Stream<SlotExecutionVertexAssignment> allocateLogicalSlotsFromSharedSlot(
+	private Map<ExecutionVertexID, SlotExecutionVertexAssignment> allocateLogicalSlotsFromSharedSlots(
 			SharedSlotProfileRetriever sharedSlotProfileRetriever,
-			ExecutionSlotSharingGroup executionSlotSharingGroup,
-			Collection<ExecutionVertexID> executions) {
-		SharedSlot sharedSlot = getOrAllocateSharedSlot(executionSlotSharingGroup, sharedSlotProfileRetriever);
-		return executions
-			.stream()
-			.map(execution -> new SlotExecutionVertexAssignment(execution, sharedSlot.allocateLogicalSlot(execution)));
+			Map<ExecutionSlotSharingGroup, List<ExecutionVertexID>> executionsByGroup) {
+		Map<ExecutionVertexID, SlotExecutionVertexAssignment> assignments = new HashMap<>();
+		for (Map.Entry<ExecutionSlotSharingGroup, List<ExecutionVertexID>> entry : executionsByGroup.entrySet()) {
+			ExecutionSlotSharingGroup group = entry.getKey();
+			List<ExecutionVertexID> executionIds = entry.getValue();
+			SharedSlot sharedSlot = getOrAllocateSharedSlot(group, sharedSlotProfileRetriever);
+			for (ExecutionVertexID executionId : executionIds) {
+				CompletableFuture<LogicalSlot> logicalSlotFuture = sharedSlot.allocateLogicalSlot(executionId);
+				SlotExecutionVertexAssignment assignment = new SlotExecutionVertexAssignment(executionId, logicalSlotFuture);
+				assignments.put(executionId, assignment);
+			}

Review comment:
       nit: Some empty lines could make it a bit more readable.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SharingPhysicalSlotRequestBulkTest.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.java.tuple.Tuple2;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+
+import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createRandomExecutionVertexId;
+import static org.apache.flink.runtime.scheduler.SharedSlotTestingUtils.createExecutionSlotSharingGroup;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.hasSize;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test suite for {@link SharingPhysicalSlotRequestBulk}.
+ */
+public class SharingPhysicalSlotRequestBulkTest {

Review comment:
       ```suggestion
   public class SharingPhysicalSlotRequestBulkTest extends TestLogger {
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotProvider.java
##########
@@ -102,20 +91,6 @@ default void start(ComponentMainThreadExecutor mainThreadExecutor) {
 			allocationTimeout);
 	}
 
-	/**
-	 * Allocates a bulk of physical slots. The allocation will be completed
-	 * normally only when all the requests are fulfilled.
-	 *
-	 * @param physicalSlotRequests requests for physical slots
-	 * @param timeout indicating how long it is accepted that the slot requests can be unfulfillable
-	 * @return future of the results of slot requests
-	 */
-	default CompletableFuture<Collection<PhysicalSlotRequest.Result>> allocatePhysicalSlots(
-		Collection<PhysicalSlotRequest> physicalSlotRequests,
-		Time timeout) {
-		throw new UnsupportedOperationException("Not properly implemented.");
-	}
-
 	/**
 	 * Cancels the slot request with the given {@link SlotRequestId} and {@link SlotSharingGroupId}.
 	 *

Review comment:
       Can default `void cancelSlotRequest(SlotRequestId slotRequestId, Throwable cause)` be removed?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SharedSlot.java
##########
@@ -38,6 +38,26 @@
 import java.util.function.Consumer;
 import java.util.function.Function;
 
+/**
+ * Shared slot implementation for the {@link SlotSharingExecutionSlotAllocator}.
+ *
+ * <p>The shared slots are owned and tracked by {@link SlotSharingExecutionSlotAllocator}.
+ * The shared slot represents a collection of {@link SingleLogicalSlot} requests which share one physical slot.
+ * The shared slot is created by the {@link SlotSharingExecutionSlotAllocator} from the physical slot request.
+ * Afterwards, {@link SlotSharingExecutionSlotAllocator} requests logical slots from the underlying physical slot
+ * for execution which share it.
+ *
+ * <p>The shared slot becomes a {@link PhysicalSlot.Payload} of its underlying physical slot
+ * once the physical slot is obtained. If the allcoated physical slot gets released then it calls back the shared slot
+ * to release the logical slots which fail their execution payloads.
+ *
+ * <p>A logical slot request can be cancelled if it is not completed yet or returned by the execution
+ * if it has been completed and given to the execution by {@link SlotSharingExecutionSlotAllocator}.
+ * If the underlying physical slot fails, it fails all logical slot requests.
+ * The failed, cancelled or returned logical slot requests are removed from the shared slot.
+ * Once the shared slot has no registered logical slot requests, it calls back its {@link SlotSharingExecutionSlotAllocator}
+ * to remove it from the allocator and cancel its underlying physical slot request if the request is not fulfilled yet.

Review comment:
       Nice :-)




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



[GitHub] [flink] zhuzhurk commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r484459944



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -161,158 +173,66 @@ private SharedSlot getOrAllocateSharedSlot(
 					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
 						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
 					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
-				return new SharedSlot(physicalSlotRequestId, physicalSlotResourceProfile, group, physicalSlotFuture);
+				return new SharedSlot(
+					physicalSlotRequestId,
+					physicalSlotResourceProfile,
+					group,
+					physicalSlotFuture,
+					slotWillBeOccupiedIndefinitely,
+					this::releaseSharedSlot);
 			});
 	}
 
+	private void releaseSharedSlot(ExecutionSlotSharingGroup executionSlotSharingGroup) {
+		SharedSlot slot = sharedSlots.remove(executionSlotSharingGroup);
+		if (slot != null) {
+			slotProvider.cancelSlotRequest(
+				slot.getPhysicalSlotRequestId(),
+				new FlinkException("Slot is being returned from SlotSharingExecutionSlotAllocator."));
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup {} to release", executionSlotSharingGroup);

Review comment:
       I think `if (!isEmpty()) {` can work.
   But then we are having a implicit assumption that `SharedSlot#release()` will not be invoked before any `SharedSlot#allocateLogicalSlot()` happens, which looks a bit tricky. 




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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * bce753ff7f4da2cffa295bd4007517af4c5697d8 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926) 
   * 18f88af3b438b13e9a240efd2b4979f841d2b978 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202",
       "triggerID" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219",
       "triggerID" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6299",
       "triggerID" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6307",
       "triggerID" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "23cbf376a6cc07924cf9101a95962a34359c1b76",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "23cbf376a6cc07924cf9101a95962a34359c1b76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8dde2eb2e2bb41c213d23f874be8392365f2c4f9",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6328",
       "triggerID" : "8dde2eb2e2bb41c213d23f874be8392365f2c4f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fe8cc1fc15e29ede254333ae186eb0b6cb62d5a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9fe8cc1fc15e29ede254333ae186eb0b6cb62d5a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dbed0f154c63a16b28530d8d1a5ff351ea64f3d0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6331",
       "triggerID" : "dbed0f154c63a16b28530d8d1a5ff351ea64f3d0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * 23cbf376a6cc07924cf9101a95962a34359c1b76 UNKNOWN
   * 8dde2eb2e2bb41c213d23f874be8392365f2c4f9 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6328) 
   * 9fe8cc1fc15e29ede254333ae186eb0b6cb62d5a UNKNOWN
   * dbed0f154c63a16b28530d8d1a5ff351ea64f3d0 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6331) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] zhuzhurk commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r479087979



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java
##########
@@ -265,12 +292,99 @@ public void testPhysicalSlotReleaseLogicalSlots() throws ExecutionException, Int
 				return payload;
 			})
 			.collect(Collectors.toList());
+		SlotRequestId slotRequestId = context.getSlotProvider().getFirstRequestOrFail().getSlotRequestId();
 		TestingPhysicalSlot physicalSlot = context.getSlotProvider().getFirstResponseOrFail().get();
 
 		assertThat(payloads.stream().allMatch(payload -> payload.getTerminalStateFuture().isDone()), is(false));
 		assertThat(physicalSlot.getPayload(), notNullValue());
 		physicalSlot.getPayload().release(new Throwable());
 		assertThat(payloads.stream().allMatch(payload -> payload.getTerminalStateFuture().isDone()), is(true));
+
+		assertThat(context.getSlotProvider().getCancellations().containsKey(slotRequestId), is(true));
+
+		context.allocateSlotsFor(EV1, EV2);
+		// there should be one more physical slot allocation, as the first allocation should be removed after releasing all logical slots
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testSchedulePendingRequestBulkTimeoutCheck() {
+		TestingPhysicalSlotRequestBulkChecker bulkChecker = new TestingPhysicalSlotRequestBulkChecker();
+		AllocationContext context = createBulkCheckerContextWithEv12GroupAndEv3Group(bulkChecker);
+
+		context.allocateSlotsFor(EV1, EV3);
+		PhysicalSlotRequestBulk bulk = bulkChecker.getBulk();
+
+		assertThat(bulk.getPendingRequests(), hasSize(2));
+		assertThat(bulk.getPendingRequests(), containsInAnyOrder(RESOURCE_PROFILE.multiply(2), RESOURCE_PROFILE));
+		assertThat(bulk.getAllocationIdsOfFulfilledRequests(), hasSize(0));
+		assertThat(bulkChecker.getTimeout(), is(ALLOCATION_TIMEOUT));
+	}
+
+	@Test
+	public void testRequestFulfilledInBulk() {
+		TestingPhysicalSlotRequestBulkChecker bulkChecker = new TestingPhysicalSlotRequestBulkChecker();
+		AllocationContext context = createBulkCheckerContextWithEv12GroupAndEv3Group(bulkChecker);
+
+		context.allocateSlotsFor(EV1, EV3);
+		AllocationID allocationId = new AllocationID();
+		ResourceProfile pendingSlotResourceProfile = fulfilOneOfTwoSlotRequestsAndGetPendingProfile(context, allocationId);
+		PhysicalSlotRequestBulk bulk = bulkChecker.getBulk();
+
+		assertThat(bulk.getPendingRequests(), hasSize(1));
+		assertThat(bulk.getPendingRequests(), containsInAnyOrder(pendingSlotResourceProfile));
+		assertThat(bulk.getAllocationIdsOfFulfilledRequests(), hasSize(1));
+		assertThat(bulk.getAllocationIdsOfFulfilledRequests(), containsInAnyOrder(allocationId));
+	}
+
+	@Test
+	public void testRequestBulkCancel() {
+		TestingPhysicalSlotRequestBulkChecker bulkChecker = new TestingPhysicalSlotRequestBulkChecker();
+		AllocationContext context = createBulkCheckerContextWithEv12GroupAndEv3Group(bulkChecker);
+
+		// allocate 2 physical slots for 2 groups
+		List<SlotExecutionVertexAssignment> assignments1 = context.allocateSlotsFor(EV1, EV3);
+		fulfilOneOfTwoSlotRequestsAndGetPendingProfile(context, new AllocationID());
+		PhysicalSlotRequestBulk bulk1 = bulkChecker.getBulk();
+		List<SlotExecutionVertexAssignment> assignments2 = context.allocateSlotsFor(EV2);
+
+		// cancelling of (EV1, EV3) releases assignments1 and only one physical slot for EV3
+		// the second physical slot is held by sharing EV2 from the next bulk
+		bulk1.cancel(new Throwable());
+
+		// return completed logical slot to clear shared slpt and release physical slot

Review comment:
       typo: `slpt` -> `slot`




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



[GitHub] [flink] tillrohrmann commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r476318798



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SchedulerImpl.java
##########
@@ -98,14 +101,15 @@ public SchedulerImpl(
 			"Scheduler is not initialized with proper main thread executor. " +
 				"Call to Scheduler.start(...) required.");
 
-		this.bulkSlotProvider = new BulkSlotProviderImpl(slotSelectionStrategy, slotPool);
+		this.slotRequestBulkChecker = PhysicalSlotRequestBulkCheckerImpl.fromSlotPool(slotPool, SystemClock.getInstance());
+		this.bulkSlotProvider = new BulkSlotProviderImpl(slotSelectionStrategy, slotPool, slotRequestBulkChecker);

Review comment:
       It is a bit unrelated because the change has been introduced before. I was wondering why do we have to touch the `SchedulerImpl` at all? Isn't this implementation only used by the non-pipelined region scheduler implementations? Maybe it would have been easier to separate the scheduler implementations in `SchedulerImpl` for the old strategies and a new `PipelinedRegionScheduler` implementation which only supports the bulk physical slot requests.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotProviderImpl.java
##########
@@ -87,9 +86,12 @@
 			SlotRequestId slotRequestId,
 			ResourceProfile resourceProfile,
 			boolean willSlotBeOccupiedIndefinitely) {
-		return willSlotBeOccupiedIndefinitely ?
-			slotPool.requestNewAllocatedSlot(slotRequestId, resourceProfile, null) :
-			slotPool.requestNewAllocatedBatchSlot(slotRequestId, resourceProfile);
+		if (willSlotBeOccupiedIndefinitely) {
+			return slotPool.requestNewAllocatedSlot(slotRequestId, resourceProfile, null);
+		} else {
+			slotPool.disableBatchSlotRequestTimeoutCheck();

Review comment:
       Why can't this happen in the constructor?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkChecker.java
##########
@@ -18,131 +18,14 @@
 
 package org.apache.flink.runtime.jobmaster.slotpool;
 
-import org.apache.flink.annotation.VisibleForTesting;
 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.jobmaster.SlotInfo;
-import org.apache.flink.util.clock.Clock;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Optional;
-import java.util.Set;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * This class helps to check the status of physical slot request bulks.
+ * This class tracks a fulfil-ability timeout of a bulk of physical slot requests.
+ *
+ * <p>The bulk gets canceled if the timeout occurs and the bulk is not fulfillable.

Review comment:
       Nit: different spelling of `fulfill`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulk.java
##########
@@ -20,60 +20,17 @@
 
 import org.apache.flink.runtime.clusterframework.types.AllocationID;
 import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
-import org.apache.flink.runtime.jobmaster.SlotRequestId;
 
 import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.stream.Collectors;
+import java.util.Set;
 
 /**
  * Represents a bulk of physical slot requests.
  */
-class PhysicalSlotRequestBulk {
+public interface PhysicalSlotRequestBulk {
+	Collection<ResourceProfile> getPendingRequests();
 
-	private final Map<SlotRequestId, ResourceProfile> pendingRequests;
+	Set<AllocationID> getAllocationIdsOfFulfilledRequests();
 
-	private final Map<SlotRequestId, AllocationID> fulfilledRequests = new HashMap<>();
-
-	private long unfulfillableTimestamp = Long.MAX_VALUE;
-
-	PhysicalSlotRequestBulk(final Collection<PhysicalSlotRequest> physicalSlotRequests) {
-		this.pendingRequests = physicalSlotRequests.stream()
-			.collect(Collectors.toMap(
-				PhysicalSlotRequest::getSlotRequestId,
-				r -> r.getSlotProfile().getPhysicalSlotResourceProfile()));
-	}
-
-	void markRequestFulfilled(final SlotRequestId slotRequestId, final AllocationID allocationID) {
-		pendingRequests.remove(slotRequestId);
-		fulfilledRequests.put(slotRequestId, allocationID);
-	}
-
-	Map<SlotRequestId, ResourceProfile> getPendingRequests() {
-		return Collections.unmodifiableMap(pendingRequests);
-	}
-
-	Map<SlotRequestId, AllocationID> getFulfilledRequests() {
-		return Collections.unmodifiableMap(fulfilledRequests);
-	}
-
-	void markFulfillable() {
-		unfulfillableTimestamp = Long.MAX_VALUE;
-	}
-
-	void markUnfulfillable(final long currentTimestamp) {
-		if (isFulfillable()) {
-			unfulfillableTimestamp = currentTimestamp;
-		}
-	}
-
-	long getUnfulfillableSince() {
-		return unfulfillableTimestamp;
-	}
-
-	private boolean isFulfillable() {
-		return unfulfillableTimestamp == Long.MAX_VALUE;
-	}
+	void cancel(Throwable cause);

Review comment:
       Interfaces should always have JavaDocs stating their contracts.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkChecker.java
##########
@@ -18,131 +18,14 @@
 
 package org.apache.flink.runtime.jobmaster.slotpool;
 
-import org.apache.flink.annotation.VisibleForTesting;
 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.jobmaster.SlotInfo;
-import org.apache.flink.util.clock.Clock;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Optional;
-import java.util.Set;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * This class helps to check the status of physical slot request bulks.
+ * This class tracks a fulfil-ability timeout of a bulk of physical slot requests.
+ *
+ * <p>The bulk gets canceled if the timeout occurs and the bulk is not fulfillable.
  */
-class PhysicalSlotRequestBulkChecker {
-
-	private final Supplier<Set<SlotInfo>> slotsRetriever;
-
-	private final Clock clock;
-
-	PhysicalSlotRequestBulkChecker(final Supplier<Set<SlotInfo>> slotsRetriever, final Clock clock) {
-		this.slotsRetriever = checkNotNull(slotsRetriever);
-		this.clock = checkNotNull(clock);
-	}
-
-	PhysicalSlotRequestBulk createPhysicalSlotRequestBulk(final Collection<PhysicalSlotRequest> physicalSlotRequests) {
-		final PhysicalSlotRequestBulk slotRequestBulk = new PhysicalSlotRequestBulk(physicalSlotRequests);
-		slotRequestBulk.markUnfulfillable(clock.relativeTimeMillis());
-
-		return slotRequestBulk;
-	}
-
-	/**
-	 * Check the slot request bulk and timeout its requests if it has been unfulfillable for too long.
-	 * @param slotRequestBulk bulk of slot requests
-	 * @param slotRequestTimeout indicates how long a pending request can be unfulfillable
-	 * @return result of the check, indicating the bulk is fulfilled, still pending, or timed out
-	 */
-	TimeoutCheckResult checkPhysicalSlotRequestBulkTimeout(
-			final PhysicalSlotRequestBulk slotRequestBulk,
-			final Time slotRequestTimeout) {
-
-		if (slotRequestBulk.getPendingRequests().isEmpty()) {
-			return TimeoutCheckResult.FULFILLED;
-		}
-
-		final boolean fulfillable = isSlotRequestBulkFulfillable(slotRequestBulk, slotsRetriever);
-		if (fulfillable) {
-			slotRequestBulk.markFulfillable();
-		} else {
-			final long currentTimestamp = clock.relativeTimeMillis();
-
-			slotRequestBulk.markUnfulfillable(currentTimestamp);
-
-			final long unfulfillableSince = slotRequestBulk.getUnfulfillableSince();
-			if (unfulfillableSince + slotRequestTimeout.toMilliseconds() <= currentTimestamp) {
-				return TimeoutCheckResult.TIMEOUT;
-			}
-		}
-
-		return TimeoutCheckResult.PENDING;
-	}
-
-	/**
-	 * Returns whether the given bulk of slot requests are possible to be fulfilled at the same time
-	 * with all the reusable slots in the slot pool. A reusable slot means the slot is available or
-	 * will not be occupied indefinitely.
-	 *
-	 * @param slotRequestBulk bulk of slot requests to check
-	 * @param slotsRetriever supplies slots to be used for the fulfill-ability check
-	 * @return true if the slot requests are possible to be fulfilled, otherwise false
-	 */
-	@VisibleForTesting
-	static boolean isSlotRequestBulkFulfillable(
-			final PhysicalSlotRequestBulk slotRequestBulk,
-			final Supplier<Set<SlotInfo>> slotsRetriever) {
-
-		final Set<AllocationID> assignedSlots = new HashSet<>(slotRequestBulk.getFulfilledRequests().values());
-		final Set<SlotInfo> reusableSlots = getReusableSlots(slotsRetriever, assignedSlots);
-		return areRequestsFulfillableWithSlots(slotRequestBulk.getPendingRequests().values(), reusableSlots);
-	}
-
-	private static Set<SlotInfo> getReusableSlots(
-			final Supplier<Set<SlotInfo>> slotsRetriever,
-			final Set<AllocationID> slotsToExclude) {
-
-		return slotsRetriever.get().stream()
-			.filter(slotInfo -> !slotInfo.willBeOccupiedIndefinitely())
-			.filter(slotInfo -> !slotsToExclude.contains(slotInfo.getAllocationId()))
-			.collect(Collectors.toSet());
-	}
-
-	private static boolean areRequestsFulfillableWithSlots(
-			final Collection<ResourceProfile> requestResourceProfiles,
-			final Set<SlotInfo> slots) {
-
-		final Set<SlotInfo> remainingSlots = new HashSet<>(slots);
-		for (ResourceProfile requestResourceProfile : requestResourceProfiles) {
-			final Optional<SlotInfo> matchedSlot = findMatchingSlotForRequest(requestResourceProfile, remainingSlots);
-			if (matchedSlot.isPresent()) {
-				remainingSlots.remove(matchedSlot.get());
-			} else {
-				return false;
-			}
-		}
-		return true;
-	}
-
-	private static Optional<SlotInfo> findMatchingSlotForRequest(
-			final ResourceProfile requestResourceProfile,
-			final Collection<SlotInfo> slots) {
-
-		return slots.stream().filter(slot -> slot.getResourceProfile().isMatching(requestResourceProfile)).findFirst();
-	}
-
-	enum TimeoutCheckResult {
-		PENDING,
-
-		FULFILLED,
-
-		TIMEOUT
-	}
+@FunctionalInterface
+public interface PhysicalSlotRequestBulkChecker {
+	void schedulePendingRequestBulkTimeoutCheck(PhysicalSlotRequestBulk bulk, Time timeout);

Review comment:
       Interfaces should always have JavaDocs stating their contracts.




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



[GitHub] [flink] tillrohrmann commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r481087221



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -108,27 +122,30 @@
 
 		SharedSlotProfileRetriever sharedSlotProfileRetriever = sharedSlotProfileRetrieverFactory
 			.createFromBulk(new HashSet<>(executionVertexIds));

Review comment:
       Why is it enough to only put `executionVertexIds` in the set of `MergingSharedSlotProfileRetriever.allBulkPriorAllocationIds`? What if the failover includes multiple pipelined regions? Won't this mean that it can happen that the first pipelined regions takes slots from a preceding one?




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



[GitHub] [flink] tillrohrmann commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r481089806



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -150,152 +167,72 @@ private SharedSlot getOrAllocateSharedSlot(
 		return sharedSlots
 			.computeIfAbsent(executionSlotSharingGroup, group -> {
 				SlotRequestId physicalSlotRequestId = new SlotRequestId();
+				ResourceProfile physicalSlotResourceProfile = getPhysicalSlotResourceProfile(group);
 				CompletableFuture<PhysicalSlot> physicalSlotFuture = sharedSlotProfileRetriever
-					.getSlotProfileFuture(group)
+					.getSlotProfileFuture(group, physicalSlotResourceProfile)
 					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
 						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
 					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
-				return new SharedSlot(physicalSlotRequestId, group, physicalSlotFuture);
+				return new SharedSlot(
+					physicalSlotRequestId,
+					physicalSlotResourceProfile,
+					group,
+					physicalSlotFuture,
+					slotWillBeOccupiedIndefinitely,
+					this::releaseSharedSlot);
 			});
 	}
 
-	private class SharedSlot implements SlotOwner, PhysicalSlot.Payload {
-		private final SlotRequestId physicalSlotRequestId;
-
-		private final ExecutionSlotSharingGroup executionSlotSharingGroup;
-
-		private final CompletableFuture<PhysicalSlot> slotContextFuture;
-
-		private final DualKeyLinkedMap<ExecutionVertexID, SlotRequestId, CompletableFuture<SingleLogicalSlot>> requestedLogicalSlots;
-
-		private SharedSlot(
-				SlotRequestId physicalSlotRequestId,
-				ExecutionSlotSharingGroup executionSlotSharingGroup,
-				CompletableFuture<PhysicalSlot> slotContextFuture) {
-			this.physicalSlotRequestId = physicalSlotRequestId;
-			this.executionSlotSharingGroup = executionSlotSharingGroup;
-			this.slotContextFuture = slotContextFuture.thenApply(physicalSlot -> {
-				Preconditions.checkState(
-					physicalSlot.tryAssignPayload(this),
-					"Unexpected physical slot payload assignment failure!");
-				return physicalSlot;
-			});
-			this.requestedLogicalSlots = new DualKeyLinkedMap<>(executionSlotSharingGroup.getExecutionVertexIds().size());
-		}
-
-		private CompletableFuture<LogicalSlot> allocateLogicalSlot(ExecutionVertexID executionVertexId) {
-			Preconditions.checkArgument(executionSlotSharingGroup.getExecutionVertexIds().contains(executionVertexId));
-			CompletableFuture<SingleLogicalSlot> logicalSlotFuture = requestedLogicalSlots.getValueByKeyA(executionVertexId);
-			if (logicalSlotFuture != null) {
-				LOG.debug("Request for {} already exists", getLogicalSlotString(executionVertexId));
-			} else {
-				logicalSlotFuture = allocateNonExistentLogicalSlot(executionVertexId);
-			}
-			return logicalSlotFuture.thenApply(Function.identity());
+	private void releaseSharedSlot(ExecutionSlotSharingGroup executionSlotSharingGroup) {
+		SharedSlot slot = sharedSlots.remove(executionSlotSharingGroup);

Review comment:
       Maybe add an assertion that the `SharedSlot` is empty (has no requested logical slots assigned).




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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r478385445



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotProviderImpl.java
##########
@@ -87,9 +86,12 @@
 			SlotRequestId slotRequestId,
 			ResourceProfile resourceProfile,
 			boolean willSlotBeOccupiedIndefinitely) {
-		return willSlotBeOccupiedIndefinitely ?
-			slotPool.requestNewAllocatedSlot(slotRequestId, resourceProfile, null) :
-			slotPool.requestNewAllocatedBatchSlot(slotRequestId, resourceProfile);
+		if (willSlotBeOccupiedIndefinitely) {
+			return slotPool.requestNewAllocatedSlot(slotRequestId, resourceProfile, null);
+		} else {
+			slotPool.disableBatchSlotRequestTimeoutCheck();

Review comment:
       This is a side effect related to the previous comment. `PhysicalSlotProviderImpl` is always created atm and we can break the legacy setup if we do it in the constructor. This should become irrelevant once we remove the legacy code.




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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202",
       "triggerID" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * d34648bab5e9a642c97db6e98f80f8059500c145 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202) 
   * db1607ead33ae1b60634c1162ba8247a9957f0f4 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r481882451



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkCheckerImpl.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.jobmaster.slotpool;
+
+import org.apache.flink.annotation.VisibleForTesting;
+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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.util.clock.Clock;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+class PhysicalSlotRequestBulkCheckerImpl implements PhysicalSlotRequestBulkChecker {
+
+	private ComponentMainThreadExecutor componentMainThreadExecutor;
+
+	private final Supplier<Set<SlotInfo>> slotsRetriever;
+
+	private final Clock clock;
+
+	PhysicalSlotRequestBulkCheckerImpl(final Supplier<Set<SlotInfo>> slotsRetriever, final Clock clock) {
+		this.slotsRetriever = checkNotNull(slotsRetriever);
+		this.clock = checkNotNull(clock);
+	}
+
+	void start(final ComponentMainThreadExecutor mainThreadExecutor) {
+		this.componentMainThreadExecutor = mainThreadExecutor;
+	}
+
+	@Override
+	public void schedulePendingRequestBulkTimeoutCheck(PhysicalSlotRequestBulk bulk, Time timeout) {
+		PhysicalSlotRequestBulkWithTimestamp bulkWithTimestamp = new PhysicalSlotRequestBulkWithTimestamp(bulk);
+		bulkWithTimestamp.markUnfulfillable(clock.relativeTimeMillis());
+		schedulePendingRequestBulkTimeoutCheck(bulkWithTimestamp, timeout);
+	}
+
+	private void schedulePendingRequestBulkTimeoutCheck(PhysicalSlotRequestBulkWithTimestamp bulk, Time timeout) {
+		componentMainThreadExecutor.schedule(() -> {
+			TimeoutCheckResult result = checkPhysicalSlotRequestBulkTimeout(bulk, timeout);
+
+			switch (result) {
+				case PENDING:
+					//re-schedule the timeout check
+					schedulePendingRequestBulkTimeoutCheck(bulk, timeout);
+					break;
+				case TIMEOUT:
+					bulk.cancel(new TimeoutException("Slot request bulk is not fulfillable!"));
+					break;
+				case FULFILLED:
+				default:
+					// no action to take
+					break;
+			}
+		}, timeout.getSize(), timeout.getUnit());
+	}
+
+	/**
+	 * Check the slot request bulk and timeout its requests if it has been unfulfillable for too long.
+	 * @param slotRequestBulk bulk of slot requests
+	 * @param slotRequestTimeout indicates how long a pending request can be unfulfillable
+	 * @return result of the check, indicating the bulk is fulfilled, still pending, or timed out
+	 */
+	@VisibleForTesting
+	TimeoutCheckResult checkPhysicalSlotRequestBulkTimeout(
+			final PhysicalSlotRequestBulkWithTimestamp slotRequestBulk,
+			final Time slotRequestTimeout) {
+
+		if (slotRequestBulk.getPendingRequests().isEmpty()) {
+			return TimeoutCheckResult.FULFILLED;
+		}
+
+		final boolean fulfillable = isSlotRequestBulkFulfillable(slotRequestBulk, slotsRetriever);
+		if (fulfillable) {
+			slotRequestBulk.markFulfillable();
+		} else {
+			final long currentTimestamp = clock.relativeTimeMillis();
+
+			slotRequestBulk.markUnfulfillable(currentTimestamp);
+
+			final long unfulfillableSince = slotRequestBulk.getUnfulfillableSince();
+			if (unfulfillableSince + slotRequestTimeout.toMilliseconds() <= currentTimestamp) {
+				return TimeoutCheckResult.TIMEOUT;
+			}
+		}
+
+		return TimeoutCheckResult.PENDING;
+	}
+
+	/**
+	 * Returns whether the given bulk of slot requests are possible to be fulfilled at the same time
+	 * with all the reusable slots in the slot pool. A reusable slot means the slot is available or
+	 * will not be occupied indefinitely.
+	 *
+	 * @param slotRequestBulk bulk of slot requests to check
+	 * @param slotsRetriever supplies slots to be used for the fulfill-ability check
+	 * @return true if the slot requests are possible to be fulfilled, otherwise false
+	 */
+	@VisibleForTesting
+	static boolean isSlotRequestBulkFulfillable(
+			final PhysicalSlotRequestBulk slotRequestBulk,
+			final Supplier<Set<SlotInfo>> slotsRetriever) {
+
+		final Set<AllocationID> assignedSlots = slotRequestBulk.getAllocationIdsOfFulfilledRequests();
+		final Set<SlotInfo> reusableSlots = getReusableSlots(slotsRetriever, assignedSlots);
+		return areRequestsFulfillableWithSlots(slotRequestBulk.getPendingRequests(), reusableSlots);
+	}
+
+	private static Set<SlotInfo> getReusableSlots(
+			final Supplier<Set<SlotInfo>> slotsRetriever,
+			final Set<AllocationID> slotsToExclude) {
+
+		return slotsRetriever.get().stream()
+			.filter(slotInfo -> !slotInfo.willBeOccupiedIndefinitely())
+			.filter(slotInfo -> !slotsToExclude.contains(slotInfo.getAllocationId()))
+			.collect(Collectors.toSet());
+	}
+
+	private static boolean areRequestsFulfillableWithSlots(
+			final Collection<ResourceProfile> requestResourceProfiles,
+			final Set<SlotInfo> slots) {
+
+		final Set<SlotInfo> remainingSlots = new HashSet<>(slots);
+		for (ResourceProfile requestResourceProfile : requestResourceProfiles) {
+			final Optional<SlotInfo> matchedSlot = findMatchingSlotForRequest(requestResourceProfile, remainingSlots);
+			if (matchedSlot.isPresent()) {
+				remainingSlots.remove(matchedSlot.get());
+			} else {
+				return false;
+			}
+		}
+		return true;
+	}
+
+	private static Optional<SlotInfo> findMatchingSlotForRequest(
+			final ResourceProfile requestResourceProfile,
+			final Collection<SlotInfo> slots) {
+
+		return slots.stream().filter(slot -> slot.getResourceProfile().isMatching(requestResourceProfile)).findFirst();
+	}
+
+	static PhysicalSlotRequestBulkCheckerImpl fromSlotPool(final SlotPool slotPool, final Clock clock) {

Review comment:
       Why not to have more descriptive name for the factory method, e.g. `createFromSlotPool` then?
   is it not the benefit of factory methods compared to constructors?




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



[GitHub] [flink] zhuzhurk commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r484629816



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SharedSlotTest.java
##########
@@ -234,6 +236,45 @@ public void testReleaseIfPhysicalSlotIsAllocated() {
 		assertThat(released.isDone(), is(true));
 	}
 
+	@Test
+	public void tesDuplicatedReturnLogicalSlotFails() {
+		CompletableFuture<PhysicalSlot> slotContextFuture = CompletableFuture
+			.completedFuture(new TestingPhysicalSlot(RP, new AllocationID()));
+		AtomicInteger released = new AtomicInteger(0);
+		SharedSlot sharedSlot = SharedSlotBuilder
+			.newBuilder()
+			.withSlotContextFuture(slotContextFuture)
+			.withReleaseCallback(g -> released.incrementAndGet())

Review comment:
       How about adding a case which will invoke `SharedSlot#release()` in the release callback?
   This can happen in production code path.




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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b54bb16026ebc704e7dbab97bba1f18e0438264b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631) 
   * 347df14dd8621cf7922f723aa5faf96a4f4fc1c9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r482052653



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SharedSlot.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.ResourceProfile;
+import org.apache.flink.runtime.jobmanager.scheduler.Locality;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotOwner;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.util.DualKeyLinkedMap;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+class SharedSlot implements SlotOwner, PhysicalSlot.Payload {
+	private static final Logger LOG = LoggerFactory.getLogger(SharedSlot.class);
+
+	private final SlotRequestId physicalSlotRequestId;
+
+	private final ResourceProfile physicalSlotResourceProfile;
+
+	private final ExecutionSlotSharingGroup executionSlotSharingGroup;
+
+	private final CompletableFuture<PhysicalSlot> slotContextFuture;
+
+	private final DualKeyLinkedMap<ExecutionVertexID, SlotRequestId, CompletableFuture<SingleLogicalSlot>> requestedLogicalSlots;
+
+	private final boolean slotWillBeOccupiedIndefinitely;
+
+	private final Consumer<ExecutionSlotSharingGroup> releaseCallback;
+
+	SharedSlot(
+			SlotRequestId physicalSlotRequestId,
+			ResourceProfile physicalSlotResourceProfile,
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			CompletableFuture<PhysicalSlot> slotContextFuture,
+			boolean slotWillBeOccupiedIndefinitely,
+			Consumer<ExecutionSlotSharingGroup> releaseCallback) {
+		this.physicalSlotRequestId = physicalSlotRequestId;
+		this.physicalSlotResourceProfile = physicalSlotResourceProfile;
+		this.executionSlotSharingGroup = executionSlotSharingGroup;
+		this.slotContextFuture = slotContextFuture.thenApply(physicalSlot -> {
+			Preconditions.checkState(
+				physicalSlot.tryAssignPayload(this),
+				"Unexpected physical slot payload assignment failure!");
+			return physicalSlot;
+		});
+		this.requestedLogicalSlots = new DualKeyLinkedMap<>(executionSlotSharingGroup.getExecutionVertexIds().size());
+		this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+		this.releaseCallback = releaseCallback;
+	}
+
+	SlotRequestId getPhysicalSlotRequestId() {
+		return physicalSlotRequestId;
+	}
+
+	ResourceProfile getPhysicalSlotResourceProfile() {
+		return physicalSlotResourceProfile;
+	}
+
+	CompletableFuture<PhysicalSlot> getSlotContextFuture() {
+		return slotContextFuture;
+	}
+
+	CompletableFuture<LogicalSlot> allocateLogicalSlot(ExecutionVertexID executionVertexId) {
+		Preconditions.checkArgument(executionSlotSharingGroup.getExecutionVertexIds().contains(executionVertexId));
+		CompletableFuture<SingleLogicalSlot> logicalSlotFuture = requestedLogicalSlots.getValueByKeyA(executionVertexId);
+		if (logicalSlotFuture != null) {
+			LOG.debug("Request for {} already exists", getLogicalSlotString(executionVertexId));
+		} else {
+			logicalSlotFuture = allocateNonExistentLogicalSlot(executionVertexId);
+		}
+		return logicalSlotFuture.thenApply(Function.identity());
+	}
+
+	private CompletableFuture<SingleLogicalSlot> allocateNonExistentLogicalSlot(ExecutionVertexID executionVertexId) {
+		CompletableFuture<SingleLogicalSlot> logicalSlotFuture;
+		SlotRequestId logicalSlotRequestId = new SlotRequestId();
+		String logMessageBase = getLogicalSlotString(logicalSlotRequestId, executionVertexId);
+		LOG.debug("Request a {}", logMessageBase);
+
+		logicalSlotFuture = slotContextFuture
+			.thenApply(physicalSlot -> {
+				LOG.debug("Allocated {}", logMessageBase);
+				return createLogicalSlot(physicalSlot, logicalSlotRequestId);
+			});
+		requestedLogicalSlots.put(executionVertexId, logicalSlotRequestId, logicalSlotFuture);
+
+		// If the physical slot request fails (slotContextFuture), it will also fail the logicalSlotFuture.
+		// Therefore, the next `exceptionally` callback will cancelLogicalSlotRequest and do the cleanup

Review comment:
       sorry the comment is wrong, I will update 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:
users@infra.apache.org



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r481887743



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SharedSlot.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.ResourceProfile;
+import org.apache.flink.runtime.jobmanager.scheduler.Locality;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotOwner;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.util.DualKeyLinkedMap;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+class SharedSlot implements SlotOwner, PhysicalSlot.Payload {
+	private static final Logger LOG = LoggerFactory.getLogger(SharedSlot.class);
+
+	private final SlotRequestId physicalSlotRequestId;
+
+	private final ResourceProfile physicalSlotResourceProfile;
+
+	private final ExecutionSlotSharingGroup executionSlotSharingGroup;
+
+	private final CompletableFuture<PhysicalSlot> slotContextFuture;
+
+	private final DualKeyLinkedMap<ExecutionVertexID, SlotRequestId, CompletableFuture<SingleLogicalSlot>> requestedLogicalSlots;
+
+	private final boolean slotWillBeOccupiedIndefinitely;
+
+	private final Consumer<ExecutionSlotSharingGroup> releaseCallback;
+
+	SharedSlot(
+			SlotRequestId physicalSlotRequestId,
+			ResourceProfile physicalSlotResourceProfile,
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			CompletableFuture<PhysicalSlot> slotContextFuture,
+			boolean slotWillBeOccupiedIndefinitely,
+			Consumer<ExecutionSlotSharingGroup> releaseCallback) {
+		this.physicalSlotRequestId = physicalSlotRequestId;
+		this.physicalSlotResourceProfile = physicalSlotResourceProfile;
+		this.executionSlotSharingGroup = executionSlotSharingGroup;
+		this.slotContextFuture = slotContextFuture.thenApply(physicalSlot -> {
+			Preconditions.checkState(
+				physicalSlot.tryAssignPayload(this),
+				"Unexpected physical slot payload assignment failure!");
+			return physicalSlot;
+		});
+		this.requestedLogicalSlots = new DualKeyLinkedMap<>(executionSlotSharingGroup.getExecutionVertexIds().size());
+		this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+		this.releaseCallback = releaseCallback;
+	}
+
+	SlotRequestId getPhysicalSlotRequestId() {
+		return physicalSlotRequestId;
+	}
+
+	ResourceProfile getPhysicalSlotResourceProfile() {
+		return physicalSlotResourceProfile;
+	}
+
+	CompletableFuture<PhysicalSlot> getSlotContextFuture() {
+		return slotContextFuture;
+	}
+
+	CompletableFuture<LogicalSlot> allocateLogicalSlot(ExecutionVertexID executionVertexId) {
+		Preconditions.checkArgument(executionSlotSharingGroup.getExecutionVertexIds().contains(executionVertexId));
+		CompletableFuture<SingleLogicalSlot> logicalSlotFuture = requestedLogicalSlots.getValueByKeyA(executionVertexId);
+		if (logicalSlotFuture != null) {
+			LOG.debug("Request for {} already exists", getLogicalSlotString(executionVertexId));
+		} else {
+			logicalSlotFuture = allocateNonExistentLogicalSlot(executionVertexId);
+		}
+		return logicalSlotFuture.thenApply(Function.identity());
+	}
+
+	private CompletableFuture<SingleLogicalSlot> allocateNonExistentLogicalSlot(ExecutionVertexID executionVertexId) {
+		CompletableFuture<SingleLogicalSlot> logicalSlotFuture;
+		SlotRequestId logicalSlotRequestId = new SlotRequestId();
+		String logMessageBase = getLogicalSlotString(logicalSlotRequestId, executionVertexId);
+		LOG.debug("Request a {}", logMessageBase);
+
+		logicalSlotFuture = slotContextFuture
+			.thenApply(physicalSlot -> {
+				LOG.debug("Allocated {}", logMessageBase);
+				return createLogicalSlot(physicalSlot, logicalSlotRequestId);
+			});
+		requestedLogicalSlots.put(executionVertexId, logicalSlotRequestId, logicalSlotFuture);
+
+		// If the physical slot request fails (slotContextFuture), it will also fail the logicalSlotFuture.
+		// Therefore, the next `exceptionally` callback will cancelLogicalSlotRequest and do the cleanup

Review comment:
       why to cancel a failed future? the cancellation will do nothing.
   if it failed, we just need to remove from bookkeeping, right?
   `cancelLogicalSlotRequest` does not remove the logical slot, just cancels the future.
   The failed future will not be canceled and will not trigger this removing callback eventually.




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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202",
       "triggerID" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * a96b2db52a0db507e0077266c8e9cb947413e1ba Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176) 
   * d34648bab5e9a642c97db6e98f80f8059500c145 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] zhuzhurk commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r484628806



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SharedSlot.java
##########
@@ -221,7 +227,14 @@ public void release(Throwable cause) {
 			logicalSlotFuture.thenAccept(logicalSlot -> logicalSlot.release(cause));
 		}
 		requestedLogicalSlots.clear();
-		releaseCallback.accept(executionSlotSharingGroup);
+		releaseExternally();
+	}
+
+	private void releaseExternally() {
+		if (state == State.ALLOCATED && requestedLogicalSlots.values().isEmpty()) {
+			externalReleaseCallback.accept(executionSlotSharingGroup);
+			state = State.RELEASED;

Review comment:
       The state needs to be updated before calling `externalReleaseCallback`.
   This is because `SharedSlot#release()` can be triggered in the calling stack.




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



[GitHub] [flink] tillrohrmann commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r480001862



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/BulkSlotProviderImpl.java
##########
@@ -169,44 +146,12 @@ public void cancelSlotRequest(SlotRequestId slotRequestId, Throwable cause) {
 		}
 	}
 
-	private void schedulePendingRequestBulkTimeoutCheck(
-			final PhysicalSlotRequestBulk slotRequestBulk,
-			final Time timeout) {
-
-		componentMainThreadExecutor.schedule(() -> {
-			final PhysicalSlotRequestBulkChecker.TimeoutCheckResult result =
-				slotRequestBulkChecker.checkPhysicalSlotRequestBulkTimeout(slotRequestBulk, timeout);
-
-			switch (result) {
-				case PENDING:
-					//re-schedule the timeout check
-					schedulePendingRequestBulkTimeoutCheck(slotRequestBulk, timeout);
-					break;
-				case TIMEOUT:
-					timeoutSlotRequestBulk(slotRequestBulk);
-					break;
-				default: // no action to take
-			}
-		}, timeout.getSize(), timeout.getUnit());
-	}
-
-	private void timeoutSlotRequestBulk(final PhysicalSlotRequestBulk slotRequestBulk) {
-		final Exception cause = new TimeoutException("Slot request bulk is not fulfillable!");
-		// pending requests must be canceled first otherwise they might be fulfilled by
-		// allocated slots released from this bulk
-		for (SlotRequestId slotRequestId : slotRequestBulk.getPendingRequests().keySet()) {
-			cancelSlotRequest(slotRequestId, cause);
-		}
-		for (SlotRequestId slotRequestId : slotRequestBulk.getFulfilledRequests().keySet()) {
-			cancelSlotRequest(slotRequestId, cause);
-		}
-	}
-
-	private Set<SlotInfo> getAllSlotInfos() {
-		return Stream
-			.concat(
-				slotPool.getAvailableSlotsInformation().stream(),
-				slotPool.getAllocatedSlotsInformation().stream())
-			.collect(Collectors.toSet());
+	private PhysicalSlotRequestBulkImpl createPhysicalSlotRequestBulk(final Collection<PhysicalSlotRequest> physicalSlotRequests) {
+		final PhysicalSlotRequestBulkImpl slotRequestBulk = new PhysicalSlotRequestBulkImpl(physicalSlotRequests
+			.stream()
+			.collect(Collectors.toMap(
+				PhysicalSlotRequest::getSlotRequestId,
+				r -> r.getSlotProfile().getPhysicalSlotResourceProfile())), this::cancelSlotRequest);

Review comment:
       nit the formatting is a bit off here
   
   ```suggestion
   		final PhysicalSlotRequestBulkImpl slotRequestBulk = new PhysicalSlotRequestBulkImpl(
   		    physicalSlotRequests
   			    .stream()
   			    .collect(Collectors.toMap(
   				    PhysicalSlotRequest::getSlotRequestId,
   				    r -> r.getSlotProfile().getPhysicalSlotResourceProfile())), 
   		    this::cancelSlotRequest);
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/BulkSlotProviderImpl.java
##########
@@ -49,50 +43,33 @@
 
 	private static final Logger LOG = LoggerFactory.getLogger(BulkSlotProviderImpl.class);
 
-	private ComponentMainThreadExecutor componentMainThreadExecutor;
-
 	private final SlotSelectionStrategy slotSelectionStrategy;
 
 	private final SlotPool slotPool;
 
 	private final PhysicalSlotRequestBulkChecker slotRequestBulkChecker;
 
-	BulkSlotProviderImpl(final SlotSelectionStrategy slotSelectionStrategy, final SlotPool slotPool) {
+	BulkSlotProviderImpl(
+			final SlotSelectionStrategy slotSelectionStrategy,
+			final SlotPool slotPool,
+			final PhysicalSlotRequestBulkChecker slotRequestBulkChecker) {
 		this.slotSelectionStrategy = checkNotNull(slotSelectionStrategy);
 		this.slotPool = checkNotNull(slotPool);
-
-		this.slotRequestBulkChecker = new PhysicalSlotRequestBulkChecker(
-			this::getAllSlotInfos,
-			SystemClock.getInstance());
-
-		this.componentMainThreadExecutor = new ComponentMainThreadExecutor.DummyComponentMainThreadExecutor(
-			"Scheduler is not initialized with proper main thread executor. " +
-				"Call to BulkSlotProvider.start(...) required.");
-	}
-
-	@Override
-	public void start(final ComponentMainThreadExecutor mainThreadExecutor) {
-		this.componentMainThreadExecutor = mainThreadExecutor;
+		this.slotRequestBulkChecker = slotRequestBulkChecker;

Review comment:
       the null checks are not consistent in this class.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkChecker.java
##########
@@ -18,131 +18,24 @@
 
 package org.apache.flink.runtime.jobmaster.slotpool;
 
-import org.apache.flink.annotation.VisibleForTesting;
 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.jobmaster.SlotInfo;
-import org.apache.flink.util.clock.Clock;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Optional;
-import java.util.Set;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * This class helps to check the status of physical slot request bulks.
+ * This class tracks a fulfil-ability timeout of a bulk of physical slot requests.
+ *
+ * <p>The check stops when all physical slot requests of the bulk are fulfilled by available or newly allocated slots.
+ * The bulk is fulfillable if all its physical slot requests can be fulfilled either by available or
+ * newly allocated slots or slots which currently used by other job subtasks.
+ * The bulk gets canceled if the timeout occurs and the bulk is not fulfillable.
+ * The timeout does not tick while the bulk is fulfillable but not fulfilled yet.

Review comment:
       tick -> trigger?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -161,158 +173,66 @@ private SharedSlot getOrAllocateSharedSlot(
 					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
 						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
 					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
-				return new SharedSlot(physicalSlotRequestId, physicalSlotResourceProfile, group, physicalSlotFuture);
+				return new SharedSlot(
+					physicalSlotRequestId,
+					physicalSlotResourceProfile,
+					group,
+					physicalSlotFuture,
+					slotWillBeOccupiedIndefinitely,
+					this::releaseSharedSlot);
 			});
 	}
 
+	private void releaseSharedSlot(ExecutionSlotSharingGroup executionSlotSharingGroup) {
+		SharedSlot slot = sharedSlots.remove(executionSlotSharingGroup);
+		if (slot != null) {
+			slotProvider.cancelSlotRequest(
+				slot.getPhysicalSlotRequestId(),
+				new FlinkException("Slot is being returned from SlotSharingExecutionSlotAllocator."));
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup {} to release", executionSlotSharingGroup);
+		}
+	}
+
 	private ResourceProfile getPhysicalSlotResourceProfile(ExecutionSlotSharingGroup executionSlotSharingGroup) {
 		return executionSlotSharingGroup
 			.getExecutionVertexIds()
 			.stream()
 			.reduce(ResourceProfile.ZERO, (r, e) -> r.merge(resourceProfileRetriever.apply(e)), ResourceProfile::merge);
 	}
 
-	private class SharedSlot implements SlotOwner, PhysicalSlot.Payload {
-		private final SlotRequestId physicalSlotRequestId;
-
-		private final ResourceProfile physicalSlotResourceProfile;
-
-		private final ExecutionSlotSharingGroup executionSlotSharingGroup;
-
-		private final CompletableFuture<PhysicalSlot> slotContextFuture;
-
-		private final DualKeyLinkedMap<ExecutionVertexID, SlotRequestId, CompletableFuture<SingleLogicalSlot>> requestedLogicalSlots;
-
-		private SharedSlot(
-				SlotRequestId physicalSlotRequestId,
-				ResourceProfile physicalSlotResourceProfile,
-				ExecutionSlotSharingGroup executionSlotSharingGroup,
-				CompletableFuture<PhysicalSlot> slotContextFuture) {
-			this.physicalSlotRequestId = physicalSlotRequestId;
-			this.physicalSlotResourceProfile = physicalSlotResourceProfile;
-			this.executionSlotSharingGroup = executionSlotSharingGroup;
-			this.slotContextFuture = slotContextFuture.thenApply(physicalSlot -> {
-				Preconditions.checkState(
-					physicalSlot.tryAssignPayload(this),
-					"Unexpected physical slot payload assignment failure!");
-				return physicalSlot;
+	private SharingPhysicalSlotRequestBulk createBulk(Map<ExecutionSlotSharingGroup, List<ExecutionVertexID>> executions) {
+		Map<ExecutionSlotSharingGroup, ResourceProfile> pendingRequests = executions
+			.keySet()
+			.stream()
+			.collect(Collectors.toMap(
+				group -> group,
+				group -> sharedSlots.get(group).getPhysicalSlotResourceProfile()
+			));
+		Map<ExecutionSlotSharingGroup, AllocationID> fulfilledRequests = new HashMap<>();
+		SharingPhysicalSlotRequestBulk bulk = new SharingPhysicalSlotRequestBulk(
+			executions,
+			pendingRequests,
+			fulfilledRequests,
+			(executionVertexId, cause) -> {
+				ExecutionSlotSharingGroup group = slotSharingStrategy.getExecutionSlotSharingGroup(executionVertexId);
+				sharedSlots.get(group).cancelLogicalSlotRequest(executionVertexId, cause);

Review comment:
       Here I would suggest to add a check state that `sharedSlots.get(group)` actually contains an item.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkCheckerImpl.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.jobmaster.slotpool;
+
+import org.apache.flink.annotation.VisibleForTesting;
+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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.util.clock.Clock;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+class PhysicalSlotRequestBulkCheckerImpl implements PhysicalSlotRequestBulkChecker {
+
+	private ComponentMainThreadExecutor componentMainThreadExecutor;
+
+	private final Supplier<Set<SlotInfo>> slotsRetriever;
+
+	private final Clock clock;
+
+	PhysicalSlotRequestBulkCheckerImpl(final Supplier<Set<SlotInfo>> slotsRetriever, final Clock clock) {
+		this.slotsRetriever = checkNotNull(slotsRetriever);
+		this.clock = checkNotNull(clock);
+	}
+
+	void start(final ComponentMainThreadExecutor mainThreadExecutor) {
+		this.componentMainThreadExecutor = mainThreadExecutor;
+	}
+
+	@Override
+	public void schedulePendingRequestBulkTimeoutCheck(PhysicalSlotRequestBulk bulk, Time timeout) {
+		PhysicalSlotRequestBulkWithTimestamp bulkWithTimestamp = new PhysicalSlotRequestBulkWithTimestamp(bulk);
+		bulkWithTimestamp.markUnfulfillable(clock.relativeTimeMillis());
+		schedulePendingRequestBulkTimeoutCheck(bulkWithTimestamp, timeout);
+	}
+
+	private void schedulePendingRequestBulkTimeoutCheck(PhysicalSlotRequestBulkWithTimestamp bulk, Time timeout) {
+		componentMainThreadExecutor.schedule(() -> {
+			TimeoutCheckResult result = checkPhysicalSlotRequestBulkTimeout(bulk, timeout);
+
+			switch (result) {
+				case PENDING:
+					//re-schedule the timeout check
+					schedulePendingRequestBulkTimeoutCheck(bulk, timeout);
+					break;
+				case TIMEOUT:
+					bulk.cancel(new TimeoutException("Slot request bulk is not fulfillable!"));
+					break;
+				case FULFILLED:
+				default:
+					// no action to take
+					break;
+			}
+		}, timeout.getSize(), timeout.getUnit());
+	}
+
+	/**
+	 * Check the slot request bulk and timeout its requests if it has been unfulfillable for too long.
+	 * @param slotRequestBulk bulk of slot requests
+	 * @param slotRequestTimeout indicates how long a pending request can be unfulfillable
+	 * @return result of the check, indicating the bulk is fulfilled, still pending, or timed out
+	 */
+	@VisibleForTesting
+	TimeoutCheckResult checkPhysicalSlotRequestBulkTimeout(
+			final PhysicalSlotRequestBulkWithTimestamp slotRequestBulk,
+			final Time slotRequestTimeout) {
+
+		if (slotRequestBulk.getPendingRequests().isEmpty()) {
+			return TimeoutCheckResult.FULFILLED;
+		}
+
+		final boolean fulfillable = isSlotRequestBulkFulfillable(slotRequestBulk, slotsRetriever);
+		if (fulfillable) {
+			slotRequestBulk.markFulfillable();
+		} else {
+			final long currentTimestamp = clock.relativeTimeMillis();
+
+			slotRequestBulk.markUnfulfillable(currentTimestamp);
+
+			final long unfulfillableSince = slotRequestBulk.getUnfulfillableSince();
+			if (unfulfillableSince + slotRequestTimeout.toMilliseconds() <= currentTimestamp) {
+				return TimeoutCheckResult.TIMEOUT;
+			}
+		}
+
+		return TimeoutCheckResult.PENDING;
+	}
+
+	/**
+	 * Returns whether the given bulk of slot requests are possible to be fulfilled at the same time
+	 * with all the reusable slots in the slot pool. A reusable slot means the slot is available or
+	 * will not be occupied indefinitely.
+	 *
+	 * @param slotRequestBulk bulk of slot requests to check
+	 * @param slotsRetriever supplies slots to be used for the fulfill-ability check
+	 * @return true if the slot requests are possible to be fulfilled, otherwise false
+	 */
+	@VisibleForTesting
+	static boolean isSlotRequestBulkFulfillable(
+			final PhysicalSlotRequestBulk slotRequestBulk,
+			final Supplier<Set<SlotInfo>> slotsRetriever) {
+
+		final Set<AllocationID> assignedSlots = slotRequestBulk.getAllocationIdsOfFulfilledRequests();
+		final Set<SlotInfo> reusableSlots = getReusableSlots(slotsRetriever, assignedSlots);
+		return areRequestsFulfillableWithSlots(slotRequestBulk.getPendingRequests(), reusableSlots);
+	}
+
+	private static Set<SlotInfo> getReusableSlots(
+			final Supplier<Set<SlotInfo>> slotsRetriever,
+			final Set<AllocationID> slotsToExclude) {
+
+		return slotsRetriever.get().stream()
+			.filter(slotInfo -> !slotInfo.willBeOccupiedIndefinitely())
+			.filter(slotInfo -> !slotsToExclude.contains(slotInfo.getAllocationId()))
+			.collect(Collectors.toSet());
+	}
+
+	private static boolean areRequestsFulfillableWithSlots(
+			final Collection<ResourceProfile> requestResourceProfiles,
+			final Set<SlotInfo> slots) {
+
+		final Set<SlotInfo> remainingSlots = new HashSet<>(slots);
+		for (ResourceProfile requestResourceProfile : requestResourceProfiles) {
+			final Optional<SlotInfo> matchedSlot = findMatchingSlotForRequest(requestResourceProfile, remainingSlots);
+			if (matchedSlot.isPresent()) {
+				remainingSlots.remove(matchedSlot.get());
+			} else {
+				return false;
+			}
+		}
+		return true;
+	}
+
+	private static Optional<SlotInfo> findMatchingSlotForRequest(
+			final ResourceProfile requestResourceProfile,
+			final Collection<SlotInfo> slots) {
+
+		return slots.stream().filter(slot -> slot.getResourceProfile().isMatching(requestResourceProfile)).findFirst();
+	}
+
+	static PhysicalSlotRequestBulkCheckerImpl fromSlotPool(final SlotPool slotPool, final Clock clock) {
+		return new PhysicalSlotRequestBulkCheckerImpl(() -> getAllSlotInfos(slotPool), clock);
+	}
+
+	private static Set<SlotInfo> getAllSlotInfos(SlotPool slotPool) {
+		return Stream
+			.concat(
+				slotPool.getAvailableSlotsInformation().stream(),
+				slotPool.getAllocatedSlotsInformation().stream())
+			.collect(Collectors.toSet());

Review comment:
       I think `slotPool.getAllocatesSlotsInformation` should already give you all slot infos.

##########
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:
       Call me old fashioned but I like to follow the KISS principle and to use the for-each loop here instead of wrapping everything in a stream and then calling a non-pure function in `forEach`. I think the streams API has its value in transforming streams via pure functions into something else but should not be used when doing computations with side effects.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -113,27 +122,30 @@
 
 		SharedSlotProfileRetriever sharedSlotProfileRetriever = sharedSlotProfileRetrieverFactory
 			.createFromBulk(new HashSet<>(executionVertexIds));
-		Map<ExecutionVertexID, SlotExecutionVertexAssignment> assignments = executionVertexIds
+		Map<ExecutionSlotSharingGroup, List<ExecutionVertexID>> executionsByGroup = executionVertexIds
 			.stream()
-			.collect(Collectors.groupingBy(slotSharingStrategy::getExecutionSlotSharingGroup))
+			.collect(Collectors.groupingBy(slotSharingStrategy::getExecutionSlotSharingGroup));
+		Map<ExecutionVertexID, SlotExecutionVertexAssignment> assignments = executionsByGroup
 			.entrySet()
 			.stream()
 			.flatMap(entry -> allocateLogicalSlotsFromSharedSlot(sharedSlotProfileRetriever, entry.getKey(), entry.getValue()))
 			.collect(Collectors.toMap(SlotExecutionVertexAssignment::getExecutionVertexId, a -> a));

Review comment:
       I am not convinced that using the stream API here is the right choice. What we are doing is to do computations with side effects. For that I would always prefer a traditional for-each loop.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkChecker.java
##########
@@ -18,131 +18,24 @@
 
 package org.apache.flink.runtime.jobmaster.slotpool;
 
-import org.apache.flink.annotation.VisibleForTesting;
 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.jobmaster.SlotInfo;
-import org.apache.flink.util.clock.Clock;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Optional;
-import java.util.Set;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * This class helps to check the status of physical slot request bulks.
+ * This class tracks a fulfil-ability timeout of a bulk of physical slot requests.

Review comment:
       Nit: I would suggest to decide whether to write `fulfil` or `fulfill` and to be consistent with spelling.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkCheckerImpl.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.jobmaster.slotpool;
+
+import org.apache.flink.annotation.VisibleForTesting;
+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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.util.clock.Clock;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+class PhysicalSlotRequestBulkCheckerImpl implements PhysicalSlotRequestBulkChecker {
+
+	private ComponentMainThreadExecutor componentMainThreadExecutor;
+
+	private final Supplier<Set<SlotInfo>> slotsRetriever;
+
+	private final Clock clock;
+
+	PhysicalSlotRequestBulkCheckerImpl(final Supplier<Set<SlotInfo>> slotsRetriever, final Clock clock) {
+		this.slotsRetriever = checkNotNull(slotsRetriever);
+		this.clock = checkNotNull(clock);
+	}
+
+	void start(final ComponentMainThreadExecutor mainThreadExecutor) {
+		this.componentMainThreadExecutor = mainThreadExecutor;
+	}
+
+	@Override
+	public void schedulePendingRequestBulkTimeoutCheck(PhysicalSlotRequestBulk bulk, Time timeout) {
+		PhysicalSlotRequestBulkWithTimestamp bulkWithTimestamp = new PhysicalSlotRequestBulkWithTimestamp(bulk);
+		bulkWithTimestamp.markUnfulfillable(clock.relativeTimeMillis());
+		schedulePendingRequestBulkTimeoutCheck(bulkWithTimestamp, timeout);
+	}
+
+	private void schedulePendingRequestBulkTimeoutCheck(PhysicalSlotRequestBulkWithTimestamp bulk, Time timeout) {
+		componentMainThreadExecutor.schedule(() -> {
+			TimeoutCheckResult result = checkPhysicalSlotRequestBulkTimeout(bulk, timeout);
+
+			switch (result) {
+				case PENDING:
+					//re-schedule the timeout check
+					schedulePendingRequestBulkTimeoutCheck(bulk, timeout);
+					break;
+				case TIMEOUT:
+					bulk.cancel(new TimeoutException("Slot request bulk is not fulfillable!"));
+					break;
+				case FULFILLED:
+				default:
+					// no action to take
+					break;
+			}
+		}, timeout.getSize(), timeout.getUnit());
+	}
+
+	/**
+	 * Check the slot request bulk and timeout its requests if it has been unfulfillable for too long.
+	 * @param slotRequestBulk bulk of slot requests
+	 * @param slotRequestTimeout indicates how long a pending request can be unfulfillable
+	 * @return result of the check, indicating the bulk is fulfilled, still pending, or timed out
+	 */
+	@VisibleForTesting
+	TimeoutCheckResult checkPhysicalSlotRequestBulkTimeout(
+			final PhysicalSlotRequestBulkWithTimestamp slotRequestBulk,
+			final Time slotRequestTimeout) {
+
+		if (slotRequestBulk.getPendingRequests().isEmpty()) {
+			return TimeoutCheckResult.FULFILLED;
+		}
+
+		final boolean fulfillable = isSlotRequestBulkFulfillable(slotRequestBulk, slotsRetriever);
+		if (fulfillable) {
+			slotRequestBulk.markFulfillable();
+		} else {
+			final long currentTimestamp = clock.relativeTimeMillis();
+
+			slotRequestBulk.markUnfulfillable(currentTimestamp);
+
+			final long unfulfillableSince = slotRequestBulk.getUnfulfillableSince();
+			if (unfulfillableSince + slotRequestTimeout.toMilliseconds() <= currentTimestamp) {
+				return TimeoutCheckResult.TIMEOUT;
+			}
+		}
+
+		return TimeoutCheckResult.PENDING;
+	}
+
+	/**
+	 * Returns whether the given bulk of slot requests are possible to be fulfilled at the same time
+	 * with all the reusable slots in the slot pool. A reusable slot means the slot is available or
+	 * will not be occupied indefinitely.
+	 *
+	 * @param slotRequestBulk bulk of slot requests to check
+	 * @param slotsRetriever supplies slots to be used for the fulfill-ability check
+	 * @return true if the slot requests are possible to be fulfilled, otherwise false
+	 */
+	@VisibleForTesting
+	static boolean isSlotRequestBulkFulfillable(
+			final PhysicalSlotRequestBulk slotRequestBulk,
+			final Supplier<Set<SlotInfo>> slotsRetriever) {
+
+		final Set<AllocationID> assignedSlots = slotRequestBulk.getAllocationIdsOfFulfilledRequests();
+		final Set<SlotInfo> reusableSlots = getReusableSlots(slotsRetriever, assignedSlots);
+		return areRequestsFulfillableWithSlots(slotRequestBulk.getPendingRequests(), reusableSlots);
+	}
+
+	private static Set<SlotInfo> getReusableSlots(
+			final Supplier<Set<SlotInfo>> slotsRetriever,
+			final Set<AllocationID> slotsToExclude) {
+
+		return slotsRetriever.get().stream()
+			.filter(slotInfo -> !slotInfo.willBeOccupiedIndefinitely())
+			.filter(slotInfo -> !slotsToExclude.contains(slotInfo.getAllocationId()))
+			.collect(Collectors.toSet());
+	}
+
+	private static boolean areRequestsFulfillableWithSlots(
+			final Collection<ResourceProfile> requestResourceProfiles,
+			final Set<SlotInfo> slots) {
+
+		final Set<SlotInfo> remainingSlots = new HashSet<>(slots);
+		for (ResourceProfile requestResourceProfile : requestResourceProfiles) {
+			final Optional<SlotInfo> matchedSlot = findMatchingSlotForRequest(requestResourceProfile, remainingSlots);
+			if (matchedSlot.isPresent()) {
+				remainingSlots.remove(matchedSlot.get());
+			} else {
+				return false;
+			}
+		}
+		return true;
+	}
+
+	private static Optional<SlotInfo> findMatchingSlotForRequest(
+			final ResourceProfile requestResourceProfile,
+			final Collection<SlotInfo> slots) {
+
+		return slots.stream().filter(slot -> slot.getResourceProfile().isMatching(requestResourceProfile)).findFirst();
+	}
+
+	static PhysicalSlotRequestBulkCheckerImpl fromSlotPool(final SlotPool slotPool, final Clock clock) {

Review comment:
       ```suggestion
   	static PhysicalSlotRequestBulkCheckerImpl create(final SlotPool slotPool, final Clock clock) {
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkCheckerImpl.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.jobmaster.slotpool;
+
+import org.apache.flink.annotation.VisibleForTesting;
+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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.util.clock.Clock;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+class PhysicalSlotRequestBulkCheckerImpl implements PhysicalSlotRequestBulkChecker {
+
+	private ComponentMainThreadExecutor componentMainThreadExecutor;
+
+	private final Supplier<Set<SlotInfo>> slotsRetriever;
+
+	private final Clock clock;
+
+	PhysicalSlotRequestBulkCheckerImpl(final Supplier<Set<SlotInfo>> slotsRetriever, final Clock clock) {
+		this.slotsRetriever = checkNotNull(slotsRetriever);
+		this.clock = checkNotNull(clock);
+	}
+
+	void start(final ComponentMainThreadExecutor mainThreadExecutor) {
+		this.componentMainThreadExecutor = mainThreadExecutor;
+	}
+
+	@Override
+	public void schedulePendingRequestBulkTimeoutCheck(PhysicalSlotRequestBulk bulk, Time timeout) {
+		PhysicalSlotRequestBulkWithTimestamp bulkWithTimestamp = new PhysicalSlotRequestBulkWithTimestamp(bulk);
+		bulkWithTimestamp.markUnfulfillable(clock.relativeTimeMillis());
+		schedulePendingRequestBulkTimeoutCheck(bulkWithTimestamp, timeout);
+	}
+
+	private void schedulePendingRequestBulkTimeoutCheck(PhysicalSlotRequestBulkWithTimestamp bulk, Time timeout) {
+		componentMainThreadExecutor.schedule(() -> {
+			TimeoutCheckResult result = checkPhysicalSlotRequestBulkTimeout(bulk, timeout);
+
+			switch (result) {
+				case PENDING:
+					//re-schedule the timeout check
+					schedulePendingRequestBulkTimeoutCheck(bulk, timeout);
+					break;
+				case TIMEOUT:
+					bulk.cancel(new TimeoutException("Slot request bulk is not fulfillable!"));
+					break;
+				case FULFILLED:
+				default:
+					// no action to take
+					break;
+			}
+		}, timeout.getSize(), timeout.getUnit());
+	}
+
+	/**
+	 * Check the slot request bulk and timeout its requests if it has been unfulfillable for too long.
+	 * @param slotRequestBulk bulk of slot requests
+	 * @param slotRequestTimeout indicates how long a pending request can be unfulfillable
+	 * @return result of the check, indicating the bulk is fulfilled, still pending, or timed out
+	 */
+	@VisibleForTesting
+	TimeoutCheckResult checkPhysicalSlotRequestBulkTimeout(
+			final PhysicalSlotRequestBulkWithTimestamp slotRequestBulk,
+			final Time slotRequestTimeout) {
+
+		if (slotRequestBulk.getPendingRequests().isEmpty()) {
+			return TimeoutCheckResult.FULFILLED;
+		}
+
+		final boolean fulfillable = isSlotRequestBulkFulfillable(slotRequestBulk, slotsRetriever);
+		if (fulfillable) {
+			slotRequestBulk.markFulfillable();
+		} else {
+			final long currentTimestamp = clock.relativeTimeMillis();
+
+			slotRequestBulk.markUnfulfillable(currentTimestamp);
+
+			final long unfulfillableSince = slotRequestBulk.getUnfulfillableSince();
+			if (unfulfillableSince + slotRequestTimeout.toMilliseconds() <= currentTimestamp) {
+				return TimeoutCheckResult.TIMEOUT;
+			}
+		}
+
+		return TimeoutCheckResult.PENDING;
+	}
+
+	/**
+	 * Returns whether the given bulk of slot requests are possible to be fulfilled at the same time
+	 * with all the reusable slots in the slot pool. A reusable slot means the slot is available or
+	 * will not be occupied indefinitely.
+	 *
+	 * @param slotRequestBulk bulk of slot requests to check
+	 * @param slotsRetriever supplies slots to be used for the fulfill-ability check
+	 * @return true if the slot requests are possible to be fulfilled, otherwise false
+	 */
+	@VisibleForTesting
+	static boolean isSlotRequestBulkFulfillable(
+			final PhysicalSlotRequestBulk slotRequestBulk,
+			final Supplier<Set<SlotInfo>> slotsRetriever) {
+
+		final Set<AllocationID> assignedSlots = slotRequestBulk.getAllocationIdsOfFulfilledRequests();
+		final Set<SlotInfo> reusableSlots = getReusableSlots(slotsRetriever, assignedSlots);
+		return areRequestsFulfillableWithSlots(slotRequestBulk.getPendingRequests(), reusableSlots);
+	}
+
+	private static Set<SlotInfo> getReusableSlots(
+			final Supplier<Set<SlotInfo>> slotsRetriever,
+			final Set<AllocationID> slotsToExclude) {
+
+		return slotsRetriever.get().stream()
+			.filter(slotInfo -> !slotInfo.willBeOccupiedIndefinitely())
+			.filter(slotInfo -> !slotsToExclude.contains(slotInfo.getAllocationId()))
+			.collect(Collectors.toSet());
+	}
+
+	private static boolean areRequestsFulfillableWithSlots(
+			final Collection<ResourceProfile> requestResourceProfiles,
+			final Set<SlotInfo> slots) {
+
+		final Set<SlotInfo> remainingSlots = new HashSet<>(slots);
+		for (ResourceProfile requestResourceProfile : requestResourceProfiles) {
+			final Optional<SlotInfo> matchedSlot = findMatchingSlotForRequest(requestResourceProfile, remainingSlots);
+			if (matchedSlot.isPresent()) {
+				remainingSlots.remove(matchedSlot.get());
+			} else {
+				return false;
+			}
+		}

Review comment:
       We don't have to address it right away but we should keep in mind that this might have a quadratic complexity.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SchedulerImpl.java
##########
@@ -98,14 +101,15 @@ public SchedulerImpl(
 			"Scheduler is not initialized with proper main thread executor. " +
 				"Call to Scheduler.start(...) required.");
 
-		this.bulkSlotProvider = new BulkSlotProviderImpl(slotSelectionStrategy, slotPool);
+		this.slotRequestBulkChecker = PhysicalSlotRequestBulkCheckerImpl.fromSlotPool(slotPool, SystemClock.getInstance());
+		this.bulkSlotProvider = new BulkSlotProviderImpl(slotSelectionStrategy, slotPool, slotRequestBulkChecker);

Review comment:
       Looking at the code, wouldn't the change be quite small? If I am not mistaken, then it would be enough to change the `schedulerFactory` argument of the `JobMaster` into a `SlotProviderFactory`. That way we should be able to provide a different `SlotProvider` implementation than `SchedulerImpl` to the `JobMaster` and we would not have to make sure that the `SchedulerImpl` implements all methods of the `SlotProvider` interface.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkCheckerImplTest.java
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.jobmaster.slotpool;
+
+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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.clock.ManualClock;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+import static org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotTestUtils.createPhysicalSlot;
+import static org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotTestUtils.occupyPhysicalSlot;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for {@link PhysicalSlotRequestBulkCheckerImpl}.
+ */
+public class PhysicalSlotRequestBulkCheckerImplTest extends TestLogger {
+
+	private static final Time TIMEOUT = Time.milliseconds(100L);

Review comment:
       Could this also be 50ms?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkWithTimestamp.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.jobmaster.slotpool;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+
+import java.util.Collection;
+import java.util.Set;
+
+class PhysicalSlotRequestBulkWithTimestamp implements PhysicalSlotRequestBulk {

Review comment:
       I think this class is being used in previous commits. Hence, the order in which the commits are split up does not work.

##########
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());

Review comment:
       `Collections.unmodifiableSet` might be a bit cheaper here.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SharedSlot.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.ResourceProfile;
+import org.apache.flink.runtime.jobmanager.scheduler.Locality;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotOwner;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.util.DualKeyLinkedMap;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+class SharedSlot implements SlotOwner, PhysicalSlot.Payload {

Review comment:
       `JavaDocs` would be good here.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkCheckerImplTest.java
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.jobmaster.slotpool;
+
+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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.clock.ManualClock;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+import static org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotTestUtils.createPhysicalSlot;
+import static org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotTestUtils.occupyPhysicalSlot;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for {@link PhysicalSlotRequestBulkCheckerImpl}.
+ */
+public class PhysicalSlotRequestBulkCheckerImplTest extends TestLogger {
+
+	private static final Time TIMEOUT = Time.milliseconds(100L);
+
+	private static ScheduledExecutorService singleThreadScheduledExecutorService;
+
+	private static ComponentMainThreadExecutor mainThreadExecutor;
+
+	private final ManualClock clock = new ManualClock();
+
+	private PhysicalSlotRequestBulkCheckerImpl bulkChecker;
+
+	private Set<PhysicalSlot> slots;
+
+	private Supplier<Set<SlotInfo>> slotsRetriever;
+
+	@BeforeClass
+	public static void setupClass() {
+		singleThreadScheduledExecutorService = Executors.newSingleThreadScheduledExecutor();
+		mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forSingleThreadExecutor(singleThreadScheduledExecutorService);
+	}
+
+	@AfterClass
+	public static void teardownClass() {
+		if (singleThreadScheduledExecutorService != null) {
+			singleThreadScheduledExecutorService.shutdownNow();
+		}
+	}
+
+	@Before
+	public void setup() throws Exception {
+		slots = new HashSet<>();
+		slotsRetriever = () -> new HashSet<>(slots);
+		bulkChecker = new PhysicalSlotRequestBulkCheckerImpl(slotsRetriever, clock);
+		bulkChecker.start(mainThreadExecutor);
+	}
+
+	@Test
+	public void testPendingBulkIsNotCancelled() throws InterruptedException, ExecutionException {
+		final CompletableFuture<SlotRequestId> cancellationFuture = new CompletableFuture<>();
+		final PhysicalSlotRequestBulk bulk = createPhysicalSlotRequestBulkWithCancellationFuture(cancellationFuture, new SlotRequestId());
+		bulkChecker.schedulePendingRequestBulkTimeoutCheck(bulk, TIMEOUT);
+		checkNotCancelledAfter(cancellationFuture, 2 * TIMEOUT.toMilliseconds());
+	}
+
+	@Test
+	public void testFulfilledBulkIsNotCancelled() throws InterruptedException, ExecutionException {
+		final CompletableFuture<SlotRequestId> cancellationFuture = new CompletableFuture<>();
+		final PhysicalSlotRequestBulk bulk = createPhysicalSlotRequestBulkWithCancellationFuture(cancellationFuture, new SlotRequestId());
+		bulkChecker.schedulePendingRequestBulkTimeoutCheck(bulk, TIMEOUT);
+		checkNotCancelledAfter(cancellationFuture, 2  * TIMEOUT.toMilliseconds());
+	}
+
+	private static void checkNotCancelledAfter(
+			CompletableFuture<?> cancellationFuture, long milli) throws ExecutionException, InterruptedException {
+		mainThreadExecutor.schedule(() -> {}, milli, TimeUnit.MILLISECONDS).get();
+		try {
+			assertThat(cancellationFuture.isDone(), is(false));
+			cancellationFuture.get(milli, TimeUnit.MILLISECONDS);
+			fail("The future must not have been cancelled");
+		} catch (TimeoutException e) {
+			assertThat(cancellationFuture.isDone(), is(false));
+		}
+	}
+
+	@Test
+	public void testUnfulfillableBulkIsCancelled() {
+		final CompletableFuture<SlotRequestId> cancellationFuture = new CompletableFuture<>();
+		final SlotRequestId slotRequestId = new SlotRequestId();
+		final PhysicalSlotRequestBulk bulk = createPhysicalSlotRequestBulkWithCancellationFuture(cancellationFuture, slotRequestId);
+		bulkChecker.schedulePendingRequestBulkTimeoutCheck(bulk, TIMEOUT);
+		clock.advanceTime(TIMEOUT.toMilliseconds() + 1L, TimeUnit.MILLISECONDS);
+		assertThat(cancellationFuture.join(), is(slotRequestId));
+	}
+
+	@Test
+	public void testBulkFulfilledOnCheck() {
+		final SlotRequestId slotRequestId = new SlotRequestId();
+		final PhysicalSlotRequestBulkImpl bulk = createPhysicalSlotRequestBulk(slotRequestId);
+
+		bulk.markRequestFulfilled(slotRequestId, new AllocationID());
+
+		final PhysicalSlotRequestBulkWithTimestamp bulkWithTimestamp = new PhysicalSlotRequestBulkWithTimestamp(bulk);
+		assertThat(checkBulkTimeout(bulkWithTimestamp), is(PhysicalSlotRequestBulkCheckerImpl.TimeoutCheckResult.FULFILLED));
+	}
+
+	@Test
+	public void testBulkTimeoutOnCheck() {
+		final PhysicalSlotRequestBulkWithTimestamp bulk = createPhysicalSlotRequestBulkWithTimestamp(new SlotRequestId());
+
+		clock.advanceTime(TIMEOUT.toMilliseconds() + 1L, TimeUnit.MILLISECONDS);
+		assertThat(checkBulkTimeout(bulk), is(PhysicalSlotRequestBulkCheckerImpl.TimeoutCheckResult.TIMEOUT));
+	}
+
+	@Test
+	public void testBulkPendingOnCheckIfFulfillable() {
+		final PhysicalSlotRequestBulkWithTimestamp bulk = createPhysicalSlotRequestBulkWithTimestamp(new SlotRequestId());
+
+		final PhysicalSlot slot = addOneSlot();
+		occupyPhysicalSlot(slot, false);
+
+		assertThat(checkBulkTimeout(bulk), is(PhysicalSlotRequestBulkCheckerImpl.TimeoutCheckResult.PENDING));
+	}
+
+	@Test
+	public void testBulkPendingOnCheckIfUnfulfillableButNotTimedOut() {
+		final PhysicalSlotRequestBulkWithTimestamp bulk = createPhysicalSlotRequestBulkWithTimestamp(new SlotRequestId());
+
+		assertThat(checkBulkTimeout(bulk), is(PhysicalSlotRequestBulkCheckerImpl.TimeoutCheckResult.PENDING));
+	}
+
+	@Test
+	public void testBulkFulfillable() {
+		final PhysicalSlotRequestBulk bulk = createPhysicalSlotRequestBulk(new SlotRequestId());
+
+		addOneSlot();
+
+		assertThat(isFulfillable(bulk), is(true));
+	}
+
+	@Test
+	public void testBulkUnfulfillableWithInsufficientSlots() {
+		final PhysicalSlotRequestBulk bulk = createPhysicalSlotRequestBulk(new SlotRequestId(), new SlotRequestId());
+
+		addOneSlot();
+
+		assertThat(isFulfillable(bulk), is(false));
+	}
+
+	@Test
+	public void testBulkUnfulfillableWithSlotAlreadyAssignedToBulk() {
+		final SlotRequestId slotRequestId = new SlotRequestId();
+		final PhysicalSlotRequestBulkImpl bulk = createPhysicalSlotRequestBulk(slotRequestId, new SlotRequestId());
+
+		final PhysicalSlot slot = addOneSlot();
+
+		bulk.markRequestFulfilled(slotRequestId, slot.getAllocationId());
+
+		assertThat(isFulfillable(bulk), is(false));
+	}
+
+	@Test
+	public void testBulkUnfulfillableWithSlotOccupiedIndefinitely() {
+		final PhysicalSlotRequestBulk bulk = createPhysicalSlotRequestBulk(new SlotRequestId(), new SlotRequestId());
+
+		final PhysicalSlot slot1 = addOneSlot();
+		addOneSlot();
+
+		occupyPhysicalSlot(slot1, true);
+
+		assertThat(isFulfillable(bulk), is(false));
+	}
+
+	@Test
+	public void testBulkFulfillableWithSlotOccupiedTemporarily() {
+		final PhysicalSlotRequestBulk bulk = createPhysicalSlotRequestBulk(new SlotRequestId(), new SlotRequestId());
+
+		final PhysicalSlot slot1 = addOneSlot();
+		addOneSlot();
+
+		occupyPhysicalSlot(slot1, false);
+
+		assertThat(isFulfillable(bulk), is(true));
+	}
+
+	private PhysicalSlotRequestBulkWithTimestamp createPhysicalSlotRequestBulkWithTimestamp(SlotRequestId... slotRequestIds) {
+		final PhysicalSlotRequestBulkWithTimestamp bulk = new PhysicalSlotRequestBulkWithTimestamp(createPhysicalSlotRequestBulk(slotRequestIds));
+		bulk.markUnfulfillable(clock.relativeTimeMillis());
+		return bulk;
+	}
+
+	private static PhysicalSlotRequestBulkImpl createPhysicalSlotRequestBulk(SlotRequestId... slotRequestIds) {
+		final TestingPhysicalSlotRequestBulkBuilder builder = TestingPhysicalSlotRequestBulkBuilder.newBuilder();
+		for (SlotRequestId slotRequestId : slotRequestIds) {
+			builder.addPendingRequest(slotRequestId, ResourceProfile.UNKNOWN);
+		}
+		return builder.buildPhysicalSlotRequestBulkImpl();
+	}
+
+	private PhysicalSlotRequestBulk createPhysicalSlotRequestBulkWithCancellationFuture(
+		CompletableFuture<SlotRequestId> cancellationFuture,
+		SlotRequestId slotRequestId) {
+		return TestingPhysicalSlotRequestBulkBuilder
+			.newBuilder()
+			.addPendingRequest(slotRequestId, ResourceProfile.UNKNOWN)
+			.setCanceller((id, t) -> cancellationFuture.complete(id))
+			.buildPhysicalSlotRequestBulkImpl();
+	}
+
+	private PhysicalSlot addOneSlot() {
+		final PhysicalSlot slot = createPhysicalSlot();
+		slots.add(slot);

Review comment:
       I think it is not guaranteed that changes to `slot` are visible within the `bulkChecker` which uses the `mainThreadExecutor` to run since the used `Set` implementation is not thread safe.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SharedSlot.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.ResourceProfile;
+import org.apache.flink.runtime.jobmanager.scheduler.Locality;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotOwner;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.util.DualKeyLinkedMap;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+class SharedSlot implements SlotOwner, PhysicalSlot.Payload {
+	private static final Logger LOG = LoggerFactory.getLogger(SharedSlot.class);
+
+	private final SlotRequestId physicalSlotRequestId;
+
+	private final ResourceProfile physicalSlotResourceProfile;
+
+	private final ExecutionSlotSharingGroup executionSlotSharingGroup;
+
+	private final CompletableFuture<PhysicalSlot> slotContextFuture;
+
+	private final DualKeyLinkedMap<ExecutionVertexID, SlotRequestId, CompletableFuture<SingleLogicalSlot>> requestedLogicalSlots;
+
+	private final boolean slotWillBeOccupiedIndefinitely;
+
+	private final Consumer<ExecutionSlotSharingGroup> releaseCallback;
+
+	SharedSlot(
+			SlotRequestId physicalSlotRequestId,
+			ResourceProfile physicalSlotResourceProfile,
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			CompletableFuture<PhysicalSlot> slotContextFuture,
+			boolean slotWillBeOccupiedIndefinitely,
+			Consumer<ExecutionSlotSharingGroup> releaseCallback) {
+		this.physicalSlotRequestId = physicalSlotRequestId;
+		this.physicalSlotResourceProfile = physicalSlotResourceProfile;
+		this.executionSlotSharingGroup = executionSlotSharingGroup;
+		this.slotContextFuture = slotContextFuture.thenApply(physicalSlot -> {
+			Preconditions.checkState(
+				physicalSlot.tryAssignPayload(this),
+				"Unexpected physical slot payload assignment failure!");
+			return physicalSlot;
+		});
+		this.requestedLogicalSlots = new DualKeyLinkedMap<>(executionSlotSharingGroup.getExecutionVertexIds().size());
+		this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+		this.releaseCallback = releaseCallback;
+	}
+
+	SlotRequestId getPhysicalSlotRequestId() {
+		return physicalSlotRequestId;
+	}
+
+	ResourceProfile getPhysicalSlotResourceProfile() {
+		return physicalSlotResourceProfile;
+	}
+
+	CompletableFuture<PhysicalSlot> getSlotContextFuture() {
+		return slotContextFuture;
+	}
+
+	CompletableFuture<LogicalSlot> allocateLogicalSlot(ExecutionVertexID executionVertexId) {
+		Preconditions.checkArgument(executionSlotSharingGroup.getExecutionVertexIds().contains(executionVertexId));
+		CompletableFuture<SingleLogicalSlot> logicalSlotFuture = requestedLogicalSlots.getValueByKeyA(executionVertexId);
+		if (logicalSlotFuture != null) {
+			LOG.debug("Request for {} already exists", getLogicalSlotString(executionVertexId));
+		} else {
+			logicalSlotFuture = allocateNonExistentLogicalSlot(executionVertexId);
+		}
+		return logicalSlotFuture.thenApply(Function.identity());
+	}
+
+	private CompletableFuture<SingleLogicalSlot> allocateNonExistentLogicalSlot(ExecutionVertexID executionVertexId) {
+		CompletableFuture<SingleLogicalSlot> logicalSlotFuture;
+		SlotRequestId logicalSlotRequestId = new SlotRequestId();
+		String logMessageBase = getLogicalSlotString(logicalSlotRequestId, executionVertexId);
+		LOG.debug("Request a {}", logMessageBase);
+
+		logicalSlotFuture = slotContextFuture
+			.thenApply(physicalSlot -> {
+				LOG.debug("Allocated {}", logMessageBase);
+				return createLogicalSlot(physicalSlot, logicalSlotRequestId);
+			});
+		requestedLogicalSlots.put(executionVertexId, logicalSlotRequestId, logicalSlotFuture);
+
+		// If the physical slot request fails (slotContextFuture), it will also fail the logicalSlotFuture.
+		// Therefore, the next `exceptionally` callback will cancelLogicalSlotRequest and do the cleanup

Review comment:
       `exceptionally` calls `removeLogicalSlotRequest` and not `cancelLogicalSlotRequest`.

##########
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));
+	}
+
+	void markFulfilled(ExecutionSlotSharingGroup group, AllocationID allocationID) {
+		pendingRequests.remove(group);
+		fulfilledRequests.put(group, allocationID);
+	}
+
+	void clear() {
+		pendingRequests.clear();

Review comment:
       Why do we only clear the `pendingRequests` and not also the fulfilled slots? Maybe state the contract of this method in the `JavaDocs`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -161,158 +173,66 @@ private SharedSlot getOrAllocateSharedSlot(
 					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
 						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
 					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
-				return new SharedSlot(physicalSlotRequestId, physicalSlotResourceProfile, group, physicalSlotFuture);
+				return new SharedSlot(
+					physicalSlotRequestId,
+					physicalSlotResourceProfile,
+					group,
+					physicalSlotFuture,
+					slotWillBeOccupiedIndefinitely,
+					this::releaseSharedSlot);
 			});
 	}
 
+	private void releaseSharedSlot(ExecutionSlotSharingGroup executionSlotSharingGroup) {
+		SharedSlot slot = sharedSlots.remove(executionSlotSharingGroup);
+		if (slot != null) {
+			slotProvider.cancelSlotRequest(
+				slot.getPhysicalSlotRequestId(),
+				new FlinkException("Slot is being returned from SlotSharingExecutionSlotAllocator."));
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup {} to release", executionSlotSharingGroup);
+		}
+	}
+
 	private ResourceProfile getPhysicalSlotResourceProfile(ExecutionSlotSharingGroup executionSlotSharingGroup) {
 		return executionSlotSharingGroup
 			.getExecutionVertexIds()
 			.stream()
 			.reduce(ResourceProfile.ZERO, (r, e) -> r.merge(resourceProfileRetriever.apply(e)), ResourceProfile::merge);
 	}
 
-	private class SharedSlot implements SlotOwner, PhysicalSlot.Payload {
-		private final SlotRequestId physicalSlotRequestId;
-
-		private final ResourceProfile physicalSlotResourceProfile;
-
-		private final ExecutionSlotSharingGroup executionSlotSharingGroup;
-
-		private final CompletableFuture<PhysicalSlot> slotContextFuture;
-
-		private final DualKeyLinkedMap<ExecutionVertexID, SlotRequestId, CompletableFuture<SingleLogicalSlot>> requestedLogicalSlots;
-
-		private SharedSlot(
-				SlotRequestId physicalSlotRequestId,
-				ResourceProfile physicalSlotResourceProfile,
-				ExecutionSlotSharingGroup executionSlotSharingGroup,
-				CompletableFuture<PhysicalSlot> slotContextFuture) {
-			this.physicalSlotRequestId = physicalSlotRequestId;
-			this.physicalSlotResourceProfile = physicalSlotResourceProfile;
-			this.executionSlotSharingGroup = executionSlotSharingGroup;
-			this.slotContextFuture = slotContextFuture.thenApply(physicalSlot -> {
-				Preconditions.checkState(
-					physicalSlot.tryAssignPayload(this),
-					"Unexpected physical slot payload assignment failure!");
-				return physicalSlot;
+	private SharingPhysicalSlotRequestBulk createBulk(Map<ExecutionSlotSharingGroup, List<ExecutionVertexID>> executions) {
+		Map<ExecutionSlotSharingGroup, ResourceProfile> pendingRequests = executions
+			.keySet()
+			.stream()
+			.collect(Collectors.toMap(
+				group -> group,
+				group -> sharedSlots.get(group).getPhysicalSlotResourceProfile()
+			));
+		Map<ExecutionSlotSharingGroup, AllocationID> fulfilledRequests = new HashMap<>();
+		SharingPhysicalSlotRequestBulk bulk = new SharingPhysicalSlotRequestBulk(
+			executions,
+			pendingRequests,
+			fulfilledRequests,

Review comment:
       Why are we passing in an empty map here?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java
##########
@@ -223,34 +232,52 @@ public void testLogicalSlotCancelsPhysicalSlotRequestAndRemovesSharedSlot() {
 			});
 	}
 
-	private static void testLogicalSlotRequestCancellation(
+	@Test
+	public void testCompletedLogicalSlotCancelationDoesNotCancelPhysicalSlotRequestAndDoesNotRemoveSharedSlot() {
+		// physical slot request is completed and completes logical requests
+		testLogicalSlotRequestCancellationOrRelease(
+			false,
+			false,
+			(context, assignment) -> {
+				context.getAllocator().cancel(assignment.getExecutionVertexId());
+				try {
+					assignment.getLogicalSlotFuture().get();
+				} catch (InterruptedException | ExecutionException e) {
+					throw new FlinkRuntimeException("Unexpected", e);
+				}
+			});
+	}
+
+	private static void testLogicalSlotRequestCancellationOrRelease(
 			boolean completePhysicalSlotFutureManually,
-			BiConsumer<AllocationContext, SlotExecutionVertexAssignment> cancelAction) {
+			boolean cancelsPhysicalSlotRequestAndRemovesSharedSlot,
+			BiConsumer<AllocationContext, SlotExecutionVertexAssignment> cancelOrReleaseAction) {

Review comment:
       Maybe make this a `BiConsumerWithException`, then we don't need the try-catch blocks.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotProviderImpl.java
##########
@@ -87,9 +86,12 @@
 			SlotRequestId slotRequestId,
 			ResourceProfile resourceProfile,
 			boolean willSlotBeOccupiedIndefinitely) {
-		return willSlotBeOccupiedIndefinitely ?
-			slotPool.requestNewAllocatedSlot(slotRequestId, resourceProfile, null) :
-			slotPool.requestNewAllocatedBatchSlot(slotRequestId, resourceProfile);
+		if (willSlotBeOccupiedIndefinitely) {
+			return slotPool.requestNewAllocatedSlot(slotRequestId, resourceProfile, null);
+		} else {
+			slotPool.disableBatchSlotRequestTimeoutCheck();

Review comment:
       Hmm, one way to solve this problem is to not use this class in the `SchedulerImpl`, right?

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

Review comment:
       This class is lacking tests.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -161,158 +173,66 @@ private SharedSlot getOrAllocateSharedSlot(
 					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
 						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
 					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
-				return new SharedSlot(physicalSlotRequestId, physicalSlotResourceProfile, group, physicalSlotFuture);
+				return new SharedSlot(
+					physicalSlotRequestId,
+					physicalSlotResourceProfile,
+					group,
+					physicalSlotFuture,
+					slotWillBeOccupiedIndefinitely,
+					this::releaseSharedSlot);
 			});
 	}
 
+	private void releaseSharedSlot(ExecutionSlotSharingGroup executionSlotSharingGroup) {
+		SharedSlot slot = sharedSlots.remove(executionSlotSharingGroup);
+		if (slot != null) {
+			slotProvider.cancelSlotRequest(
+				slot.getPhysicalSlotRequestId(),
+				new FlinkException("Slot is being returned from SlotSharingExecutionSlotAllocator."));
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup {} to release", executionSlotSharingGroup);

Review comment:
       What are the conditions under which this branch can happen? Can't we say that there must be a `SharedSlot` when this method is called? Making this stricter could allow us to catch other programming errors faster.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SharedSlot.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.ResourceProfile;
+import org.apache.flink.runtime.jobmanager.scheduler.Locality;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotOwner;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.util.DualKeyLinkedMap;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+class SharedSlot implements SlotOwner, PhysicalSlot.Payload {

Review comment:
       This class is lacking tests.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SharedSlot.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.ResourceProfile;
+import org.apache.flink.runtime.jobmanager.scheduler.Locality;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotOwner;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.util.DualKeyLinkedMap;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+class SharedSlot implements SlotOwner, PhysicalSlot.Payload {
+	private static final Logger LOG = LoggerFactory.getLogger(SharedSlot.class);
+
+	private final SlotRequestId physicalSlotRequestId;
+
+	private final ResourceProfile physicalSlotResourceProfile;
+
+	private final ExecutionSlotSharingGroup executionSlotSharingGroup;
+
+	private final CompletableFuture<PhysicalSlot> slotContextFuture;
+
+	private final DualKeyLinkedMap<ExecutionVertexID, SlotRequestId, CompletableFuture<SingleLogicalSlot>> requestedLogicalSlots;
+
+	private final boolean slotWillBeOccupiedIndefinitely;
+
+	private final Consumer<ExecutionSlotSharingGroup> releaseCallback;
+
+	SharedSlot(
+			SlotRequestId physicalSlotRequestId,
+			ResourceProfile physicalSlotResourceProfile,
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			CompletableFuture<PhysicalSlot> slotContextFuture,
+			boolean slotWillBeOccupiedIndefinitely,
+			Consumer<ExecutionSlotSharingGroup> releaseCallback) {
+		this.physicalSlotRequestId = physicalSlotRequestId;
+		this.physicalSlotResourceProfile = physicalSlotResourceProfile;
+		this.executionSlotSharingGroup = executionSlotSharingGroup;
+		this.slotContextFuture = slotContextFuture.thenApply(physicalSlot -> {
+			Preconditions.checkState(
+				physicalSlot.tryAssignPayload(this),
+				"Unexpected physical slot payload assignment failure!");
+			return physicalSlot;
+		});
+		this.requestedLogicalSlots = new DualKeyLinkedMap<>(executionSlotSharingGroup.getExecutionVertexIds().size());
+		this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+		this.releaseCallback = releaseCallback;
+	}
+
+	SlotRequestId getPhysicalSlotRequestId() {
+		return physicalSlotRequestId;
+	}
+
+	ResourceProfile getPhysicalSlotResourceProfile() {
+		return physicalSlotResourceProfile;
+	}
+
+	CompletableFuture<PhysicalSlot> getSlotContextFuture() {
+		return slotContextFuture;
+	}
+
+	CompletableFuture<LogicalSlot> allocateLogicalSlot(ExecutionVertexID executionVertexId) {
+		Preconditions.checkArgument(executionSlotSharingGroup.getExecutionVertexIds().contains(executionVertexId));

Review comment:
       An error message could be helpful.

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

Review comment:
       A bit of `JavaDoc` explaining the purpose of this class could be helpful.

##########
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:
       Moreover, that way it would also be more explicit to enforce the contract you have stated in the comment above.




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



[GitHub] [flink] zhuzhurk commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r474532534



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TestingPhysicalSlotRequestBulkBuilder.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.jobmaster.slotpool;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequestBulkCheckerImpl.PhysicalSlotRequestBulkWithTimestamp;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.function.BiConsumer;
+
+class TestingPhysicalSlotRequestBulkBuilder {
+	private static final BiConsumer<SlotRequestId, Throwable> EMPTY_CANCELLER = (r, t) -> {};
+	private Map<SlotRequestId, ResourceProfile> pendingRequests = new HashMap<>();
+	private BiConsumer<SlotRequestId, Throwable> canceller = EMPTY_CANCELLER;
+
+	TestingPhysicalSlotRequestBulkBuilder addPendingRequests(SlotRequestId slotRequestId, ResourceProfile resourceProfile) {

Review comment:
       `addPendingRequests` -> `addPendingRequest`

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkCheckerImplTest.java
##########
@@ -0,0 +1,274 @@
+/*
+ * 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.jobmaster.slotpool;
+
+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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequestBulkCheckerImpl.PhysicalSlotRequestBulkWithTimestamp;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.clock.ManualClock;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+import static org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotTestUtils.createPhysicalSlot;
+import static org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotTestUtils.occupyPhysicalSlot;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for {@link PhysicalSlotRequestBulkCheckerImpl}.
+ */
+public class PhysicalSlotRequestBulkCheckerImplTest extends TestLogger {
+
+	private static final Time TIMEOUT = Time.milliseconds(100L);
+
+	private static ScheduledExecutorService singleThreadScheduledExecutorService;
+
+	private static ComponentMainThreadExecutor mainThreadExecutor;
+
+	private final ManualClock clock = new ManualClock();
+
+	private PhysicalSlotRequestBulkCheckerImpl bulkChecker;
+
+	private Set<PhysicalSlot> slots;
+
+	private Supplier<Set<SlotInfo>> slotsRetriever;
+
+	@BeforeClass
+	public static void setupClass() {
+		singleThreadScheduledExecutorService = Executors.newSingleThreadScheduledExecutor();
+		mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forSingleThreadExecutor(singleThreadScheduledExecutorService);
+	}
+
+	@AfterClass
+	public static void teardownClass() {
+		if (singleThreadScheduledExecutorService != null) {
+			singleThreadScheduledExecutorService.shutdownNow();
+		}
+	}
+
+	@Before
+	public void setup() throws Exception {
+		slots = new HashSet<>();
+		slotsRetriever = () -> new HashSet<>(slots);
+		bulkChecker = new PhysicalSlotRequestBulkCheckerImpl(slotsRetriever, clock);
+		bulkChecker.start(mainThreadExecutor);
+	}
+
+	@Test
+	public void testPendingBulkIsNotCancelled() throws InterruptedException, ExecutionException {
+		final PhysicalSlotRequestBulkWithCancellationTracking bulk = new PhysicalSlotRequestBulkWithCancellationTracking(ResourceProfile.ANY);
+		bulkChecker.schedulePendingRequestBulkTimeoutCheck(bulk, TIMEOUT);
+		bulk.checkNotCancelledAfter(TIMEOUT.toMilliseconds() + 10);
+	}
+
+	@Test
+	public void testFulfilledBulkIsNotCancelled() throws InterruptedException, ExecutionException {
+		final PhysicalSlotRequestBulkWithCancellationTracking bulk = new PhysicalSlotRequestBulkWithCancellationTracking();
+		bulkChecker.schedulePendingRequestBulkTimeoutCheck(bulk, TIMEOUT);
+		bulk.checkNotCancelledAfter(2  * TIMEOUT.toMilliseconds() + 10);
+	}
+
+	@Test
+	public void testUnfulfilledBulkIsCancelled() {

Review comment:
       `testUnfulfillableBulkIsCancelled ` seems more accurate because `PENDING` requests are also unfulfilled but fulfillable.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java
##########
@@ -274,6 +277,73 @@ public void testPhysicalSlotReleaseLogicalSlots() throws ExecutionException, Int
 		assertThat(payloads.stream().allMatch(payload -> payload.getTerminalStateFuture().isDone()), is(true));
 	}
 
+	@Test
+	public void testSchedulePendingRequestBulkTimeoutCheck() {
+		TestingPhysicalSlotRequestBulkChecker bulkChecker = new TestingPhysicalSlotRequestBulkChecker();
+		AllocationContext context = createBulkCheckerContextWithEv12GroupAndEv3Group(bulkChecker);
+
+		context.allocateSlotsFor(EV1, EV3);
+		PhysicalSlotRequestBulk bulk = bulkChecker.getBulk();
+
+		assertThat(bulk.getPendingRequests(), hasSize(2));
+		assertThat(bulk.getPendingRequests(), containsInAnyOrder(RESOURCE_PROFILE.multiply(2), RESOURCE_PROFILE));
+		assertThat(bulk.getAllocationIdsOfFulfilledRequests(), hasSize(0));
+		assertThat(bulkChecker.getTimeout(), is(ALLOCATION_TIMEOUT));
+	}
+
+	@Test
+	public void testRequestFulfilledInBulk() {
+		TestingPhysicalSlotRequestBulkChecker bulkChecker = new TestingPhysicalSlotRequestBulkChecker();
+		AllocationContext context = createBulkCheckerContextWithEv12GroupAndEv3Group(bulkChecker);
+
+		context.allocateSlotsFor(EV1, EV3);
+		AllocationID allocationId = new AllocationID();
+		ResourceProfile pendingSlotResourceProfile = fulfilOneOfTwoSlotRequestsAndGetPendingProfile(context, allocationId);
+		PhysicalSlotRequestBulk bulk = bulkChecker.getBulk();
+
+		assertThat(bulk.getPendingRequests(), hasSize(1));
+		assertThat(bulk.getPendingRequests(), containsInAnyOrder(pendingSlotResourceProfile));
+		assertThat(bulk.getAllocationIdsOfFulfilledRequests(), hasSize(1));
+		assertThat(bulk.getAllocationIdsOfFulfilledRequests(), containsInAnyOrder(allocationId));
+	}
+
+	@Test
+	public void testRequestBulkCancel() {
+		TestingPhysicalSlotRequestBulkChecker bulkChecker = new TestingPhysicalSlotRequestBulkChecker();
+		AllocationContext context = createBulkCheckerContextWithEv12GroupAndEv3Group(bulkChecker);
+
+		// allocate 2 physical slots for 2 groups
+		List<SlotExecutionVertexAssignment> assignments1 = context.allocateSlotsFor(EV1, EV3);
+		fulfilOneOfTwoSlotRequestsAndGetPendingProfile(context, new AllocationID());
+		PhysicalSlotRequestBulk bulk1 = bulkChecker.getBulk();
+		List<SlotExecutionVertexAssignment> assignments2 = context.allocateSlotsFor(EV2);
+		// cancelling of (EV1, EV3) releases assignments1 and only one physical slot for EV3
+		// the second physical slot is held by sharing EV2 from the next bulk
+		bulk1.cancel(new Throwable());
+		// EV3 needs again a physical slot, therefore there are 3 requests overall
+		context.allocateSlotsFor(EV1, EV3);
+		boolean ev1failed = assignments1.get(0).getLogicalSlotFuture().isCompletedExceptionally();
+		boolean ev2failed = assignments1.get(0).getLogicalSlotFuture().isCompletedExceptionally();

Review comment:
       ```suggestion
   		boolean ev3failed = assignments1.get(1).getLogicalSlotFuture().isCompletedExceptionally();
   ```

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java
##########
@@ -274,6 +277,73 @@ public void testPhysicalSlotReleaseLogicalSlots() throws ExecutionException, Int
 		assertThat(payloads.stream().allMatch(payload -> payload.getTerminalStateFuture().isDone()), is(true));
 	}
 
+	@Test
+	public void testSchedulePendingRequestBulkTimeoutCheck() {
+		TestingPhysicalSlotRequestBulkChecker bulkChecker = new TestingPhysicalSlotRequestBulkChecker();
+		AllocationContext context = createBulkCheckerContextWithEv12GroupAndEv3Group(bulkChecker);
+
+		context.allocateSlotsFor(EV1, EV3);
+		PhysicalSlotRequestBulk bulk = bulkChecker.getBulk();
+
+		assertThat(bulk.getPendingRequests(), hasSize(2));
+		assertThat(bulk.getPendingRequests(), containsInAnyOrder(RESOURCE_PROFILE.multiply(2), RESOURCE_PROFILE));
+		assertThat(bulk.getAllocationIdsOfFulfilledRequests(), hasSize(0));
+		assertThat(bulkChecker.getTimeout(), is(ALLOCATION_TIMEOUT));
+	}
+
+	@Test
+	public void testRequestFulfilledInBulk() {
+		TestingPhysicalSlotRequestBulkChecker bulkChecker = new TestingPhysicalSlotRequestBulkChecker();
+		AllocationContext context = createBulkCheckerContextWithEv12GroupAndEv3Group(bulkChecker);
+
+		context.allocateSlotsFor(EV1, EV3);
+		AllocationID allocationId = new AllocationID();
+		ResourceProfile pendingSlotResourceProfile = fulfilOneOfTwoSlotRequestsAndGetPendingProfile(context, allocationId);
+		PhysicalSlotRequestBulk bulk = bulkChecker.getBulk();
+
+		assertThat(bulk.getPendingRequests(), hasSize(1));
+		assertThat(bulk.getPendingRequests(), containsInAnyOrder(pendingSlotResourceProfile));
+		assertThat(bulk.getAllocationIdsOfFulfilledRequests(), hasSize(1));
+		assertThat(bulk.getAllocationIdsOfFulfilledRequests(), containsInAnyOrder(allocationId));
+	}
+
+	@Test
+	public void testRequestBulkCancel() {
+		TestingPhysicalSlotRequestBulkChecker bulkChecker = new TestingPhysicalSlotRequestBulkChecker();
+		AllocationContext context = createBulkCheckerContextWithEv12GroupAndEv3Group(bulkChecker);
+
+		// allocate 2 physical slots for 2 groups
+		List<SlotExecutionVertexAssignment> assignments1 = context.allocateSlotsFor(EV1, EV3);
+		fulfilOneOfTwoSlotRequestsAndGetPendingProfile(context, new AllocationID());
+		PhysicalSlotRequestBulk bulk1 = bulkChecker.getBulk();
+		List<SlotExecutionVertexAssignment> assignments2 = context.allocateSlotsFor(EV2);
+		// cancelling of (EV1, EV3) releases assignments1 and only one physical slot for EV3
+		// the second physical slot is held by sharing EV2 from the next bulk
+		bulk1.cancel(new Throwable());
+		// EV3 needs again a physical slot, therefore there are 3 requests overall
+		context.allocateSlotsFor(EV1, EV3);
+		boolean ev1failed = assignments1.get(0).getLogicalSlotFuture().isCompletedExceptionally();
+		boolean ev2failed = assignments1.get(0).getLogicalSlotFuture().isCompletedExceptionally();
+
+		assertThat(context.getSlotProvider().getRequests().values(), hasSize(3));
+		// either EV1 or EV3 logical slot future is fulfilled before cancellation
+		assertThat(ev1failed != ev2failed, is(false));
+		assertThat(assignments2.get(0).getLogicalSlotFuture().isCompletedExceptionally(), is(false));

Review comment:
       maybe also verify that `assignments2.get(0).getLogicalSlotFuture().isDone() == true` ?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkCheckerImplTest.java
##########
@@ -0,0 +1,274 @@
+/*
+ * 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.jobmaster.slotpool;
+
+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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequestBulkCheckerImpl.PhysicalSlotRequestBulkWithTimestamp;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.clock.ManualClock;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+import static org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotTestUtils.createPhysicalSlot;
+import static org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotTestUtils.occupyPhysicalSlot;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for {@link PhysicalSlotRequestBulkCheckerImpl}.
+ */
+public class PhysicalSlotRequestBulkCheckerImplTest extends TestLogger {
+
+	private static final Time TIMEOUT = Time.milliseconds(100L);
+
+	private static ScheduledExecutorService singleThreadScheduledExecutorService;
+
+	private static ComponentMainThreadExecutor mainThreadExecutor;
+
+	private final ManualClock clock = new ManualClock();
+
+	private PhysicalSlotRequestBulkCheckerImpl bulkChecker;
+
+	private Set<PhysicalSlot> slots;
+
+	private Supplier<Set<SlotInfo>> slotsRetriever;
+
+	@BeforeClass
+	public static void setupClass() {
+		singleThreadScheduledExecutorService = Executors.newSingleThreadScheduledExecutor();
+		mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forSingleThreadExecutor(singleThreadScheduledExecutorService);
+	}
+
+	@AfterClass
+	public static void teardownClass() {
+		if (singleThreadScheduledExecutorService != null) {
+			singleThreadScheduledExecutorService.shutdownNow();
+		}
+	}
+
+	@Before
+	public void setup() throws Exception {
+		slots = new HashSet<>();
+		slotsRetriever = () -> new HashSet<>(slots);
+		bulkChecker = new PhysicalSlotRequestBulkCheckerImpl(slotsRetriever, clock);
+		bulkChecker.start(mainThreadExecutor);
+	}
+
+	@Test
+	public void testPendingBulkIsNotCancelled() throws InterruptedException, ExecutionException {
+		final PhysicalSlotRequestBulkWithCancellationTracking bulk = new PhysicalSlotRequestBulkWithCancellationTracking(ResourceProfile.ANY);
+		bulkChecker.schedulePendingRequestBulkTimeoutCheck(bulk, TIMEOUT);
+		bulk.checkNotCancelledAfter(TIMEOUT.toMilliseconds() + 10);

Review comment:
       I think a small wait time is not very stable. Although the test would not fail in this case but would just not have executed the fulfillability check yet.
   I do not have a good ides yet. One workaround is to increase the timeout to several seconds to make to more stable.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkCheckerImplTest.java
##########
@@ -0,0 +1,274 @@
+/*
+ * 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.jobmaster.slotpool;
+
+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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequestBulkCheckerImpl.PhysicalSlotRequestBulkWithTimestamp;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.clock.ManualClock;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+import static org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotTestUtils.createPhysicalSlot;
+import static org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotTestUtils.occupyPhysicalSlot;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for {@link PhysicalSlotRequestBulkCheckerImpl}.
+ */
+public class PhysicalSlotRequestBulkCheckerImplTest extends TestLogger {
+
+	private static final Time TIMEOUT = Time.milliseconds(100L);
+
+	private static ScheduledExecutorService singleThreadScheduledExecutorService;
+
+	private static ComponentMainThreadExecutor mainThreadExecutor;
+
+	private final ManualClock clock = new ManualClock();
+
+	private PhysicalSlotRequestBulkCheckerImpl bulkChecker;
+
+	private Set<PhysicalSlot> slots;
+
+	private Supplier<Set<SlotInfo>> slotsRetriever;
+
+	@BeforeClass
+	public static void setupClass() {
+		singleThreadScheduledExecutorService = Executors.newSingleThreadScheduledExecutor();
+		mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forSingleThreadExecutor(singleThreadScheduledExecutorService);
+	}
+
+	@AfterClass
+	public static void teardownClass() {
+		if (singleThreadScheduledExecutorService != null) {
+			singleThreadScheduledExecutorService.shutdownNow();
+		}
+	}
+
+	@Before
+	public void setup() throws Exception {
+		slots = new HashSet<>();
+		slotsRetriever = () -> new HashSet<>(slots);
+		bulkChecker = new PhysicalSlotRequestBulkCheckerImpl(slotsRetriever, clock);
+		bulkChecker.start(mainThreadExecutor);
+	}
+
+	@Test
+	public void testPendingBulkIsNotCancelled() throws InterruptedException, ExecutionException {
+		final PhysicalSlotRequestBulkWithCancellationTracking bulk = new PhysicalSlotRequestBulkWithCancellationTracking(ResourceProfile.ANY);

Review comment:
       I think `ResourceProfile.ANY` is used for slot size only. For resource requirements, we can use `ResourceProfile.UNKNOWN`.




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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r481278689



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkCheckerImpl.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.jobmaster.slotpool;
+
+import org.apache.flink.annotation.VisibleForTesting;
+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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.util.clock.Clock;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+class PhysicalSlotRequestBulkCheckerImpl implements PhysicalSlotRequestBulkChecker {
+
+	private ComponentMainThreadExecutor componentMainThreadExecutor;
+
+	private final Supplier<Set<SlotInfo>> slotsRetriever;
+
+	private final Clock clock;
+
+	PhysicalSlotRequestBulkCheckerImpl(final Supplier<Set<SlotInfo>> slotsRetriever, final Clock clock) {
+		this.slotsRetriever = checkNotNull(slotsRetriever);
+		this.clock = checkNotNull(clock);
+	}
+
+	void start(final ComponentMainThreadExecutor mainThreadExecutor) {
+		this.componentMainThreadExecutor = mainThreadExecutor;
+	}
+
+	@Override
+	public void schedulePendingRequestBulkTimeoutCheck(PhysicalSlotRequestBulk bulk, Time timeout) {
+		PhysicalSlotRequestBulkWithTimestamp bulkWithTimestamp = new PhysicalSlotRequestBulkWithTimestamp(bulk);
+		bulkWithTimestamp.markUnfulfillable(clock.relativeTimeMillis());
+		schedulePendingRequestBulkTimeoutCheck(bulkWithTimestamp, timeout);
+	}
+
+	private void schedulePendingRequestBulkTimeoutCheck(PhysicalSlotRequestBulkWithTimestamp bulk, Time timeout) {
+		componentMainThreadExecutor.schedule(() -> {
+			TimeoutCheckResult result = checkPhysicalSlotRequestBulkTimeout(bulk, timeout);
+
+			switch (result) {
+				case PENDING:
+					//re-schedule the timeout check
+					schedulePendingRequestBulkTimeoutCheck(bulk, timeout);
+					break;
+				case TIMEOUT:
+					bulk.cancel(new TimeoutException("Slot request bulk is not fulfillable!"));
+					break;
+				case FULFILLED:
+				default:
+					// no action to take
+					break;
+			}
+		}, timeout.getSize(), timeout.getUnit());
+	}
+
+	/**
+	 * Check the slot request bulk and timeout its requests if it has been unfulfillable for too long.
+	 * @param slotRequestBulk bulk of slot requests
+	 * @param slotRequestTimeout indicates how long a pending request can be unfulfillable
+	 * @return result of the check, indicating the bulk is fulfilled, still pending, or timed out
+	 */
+	@VisibleForTesting
+	TimeoutCheckResult checkPhysicalSlotRequestBulkTimeout(
+			final PhysicalSlotRequestBulkWithTimestamp slotRequestBulk,
+			final Time slotRequestTimeout) {
+
+		if (slotRequestBulk.getPendingRequests().isEmpty()) {
+			return TimeoutCheckResult.FULFILLED;
+		}
+
+		final boolean fulfillable = isSlotRequestBulkFulfillable(slotRequestBulk, slotsRetriever);
+		if (fulfillable) {
+			slotRequestBulk.markFulfillable();
+		} else {
+			final long currentTimestamp = clock.relativeTimeMillis();
+
+			slotRequestBulk.markUnfulfillable(currentTimestamp);
+
+			final long unfulfillableSince = slotRequestBulk.getUnfulfillableSince();
+			if (unfulfillableSince + slotRequestTimeout.toMilliseconds() <= currentTimestamp) {
+				return TimeoutCheckResult.TIMEOUT;
+			}
+		}
+
+		return TimeoutCheckResult.PENDING;
+	}
+
+	/**
+	 * Returns whether the given bulk of slot requests are possible to be fulfilled at the same time
+	 * with all the reusable slots in the slot pool. A reusable slot means the slot is available or
+	 * will not be occupied indefinitely.
+	 *
+	 * @param slotRequestBulk bulk of slot requests to check
+	 * @param slotsRetriever supplies slots to be used for the fulfill-ability check
+	 * @return true if the slot requests are possible to be fulfilled, otherwise false
+	 */
+	@VisibleForTesting
+	static boolean isSlotRequestBulkFulfillable(
+			final PhysicalSlotRequestBulk slotRequestBulk,
+			final Supplier<Set<SlotInfo>> slotsRetriever) {
+
+		final Set<AllocationID> assignedSlots = slotRequestBulk.getAllocationIdsOfFulfilledRequests();
+		final Set<SlotInfo> reusableSlots = getReusableSlots(slotsRetriever, assignedSlots);
+		return areRequestsFulfillableWithSlots(slotRequestBulk.getPendingRequests(), reusableSlots);
+	}
+
+	private static Set<SlotInfo> getReusableSlots(
+			final Supplier<Set<SlotInfo>> slotsRetriever,
+			final Set<AllocationID> slotsToExclude) {
+
+		return slotsRetriever.get().stream()
+			.filter(slotInfo -> !slotInfo.willBeOccupiedIndefinitely())
+			.filter(slotInfo -> !slotsToExclude.contains(slotInfo.getAllocationId()))
+			.collect(Collectors.toSet());
+	}
+
+	private static boolean areRequestsFulfillableWithSlots(
+			final Collection<ResourceProfile> requestResourceProfiles,
+			final Set<SlotInfo> slots) {
+
+		final Set<SlotInfo> remainingSlots = new HashSet<>(slots);
+		for (ResourceProfile requestResourceProfile : requestResourceProfiles) {
+			final Optional<SlotInfo> matchedSlot = findMatchingSlotForRequest(requestResourceProfile, remainingSlots);
+			if (matchedSlot.isPresent()) {
+				remainingSlots.remove(matchedSlot.get());
+			} else {
+				return false;
+			}
+		}

Review comment:
       yes




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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202",
       "triggerID" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219",
       "triggerID" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6299",
       "triggerID" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6307",
       "triggerID" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * d01d63a8d22b82b6b8168d3abed0505191d655e6 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6307) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b54bb16026ebc704e7dbab97bba1f18e0438264b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631) 
   * 347df14dd8621cf7922f723aa5faf96a4f4fc1c9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r478384390



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SchedulerImpl.java
##########
@@ -98,14 +101,15 @@ public SchedulerImpl(
 			"Scheduler is not initialized with proper main thread executor. " +
 				"Call to Scheduler.start(...) required.");
 
-		this.bulkSlotProvider = new BulkSlotProviderImpl(slotSelectionStrategy, slotPool);
+		this.slotRequestBulkChecker = PhysicalSlotRequestBulkCheckerImpl.fromSlotPool(slotPool, SystemClock.getInstance());
+		this.bulkSlotProvider = new BulkSlotProviderImpl(slotSelectionStrategy, slotPool, slotRequestBulkChecker);

Review comment:
       yes, I also think we will need to do something like this. So far we did not see any quick way to do it. We would have to refactor factories structure. We should plan for this when we remove the legacy code.




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



[GitHub] [flink] azagrebin commented on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-689362226


   Thanks for the reviews @tillrohrmann and @zhuzhurk 
   merging this


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202",
       "triggerID" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219",
       "triggerID" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6299",
       "triggerID" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * 7b527d1937d410eaa7de59479915074ca02e6fbd Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6299) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * a96b2db52a0db507e0077266c8e9cb947413e1ba Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7e07e70b62850e70644e35b18c9168426371862c Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] zhuzhurk commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r474428620



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkCheckerImpl.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.jobmaster.slotpool;
+
+import org.apache.flink.annotation.VisibleForTesting;
+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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.util.clock.Clock;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+class PhysicalSlotRequestBulkCheckerImpl implements PhysicalSlotRequestBulkChecker {
+
+	private ComponentMainThreadExecutor componentMainThreadExecutor;
+
+	private final Supplier<Set<SlotInfo>> slotsRetriever;
+
+	private final Clock clock;
+
+	PhysicalSlotRequestBulkCheckerImpl(final Supplier<Set<SlotInfo>> slotsRetriever, final Clock clock) {
+		this.slotsRetriever = checkNotNull(slotsRetriever);
+		this.clock = checkNotNull(clock);
+	}
+
+	void start(final ComponentMainThreadExecutor mainThreadExecutor) {
+		this.componentMainThreadExecutor = mainThreadExecutor;
+	}
+
+	@Override
+	public void schedulePendingRequestBulkTimeoutCheck(PhysicalSlotRequestBulk bulk, Time timeout) {
+		schedulePendingRequestBulkTimeoutCheck(new PhysicalSlotRequestBulkWithTimestamp(bulk), timeout);
+	}
+
+	private void schedulePendingRequestBulkTimeoutCheck(PhysicalSlotRequestBulkWithTimestamp bulk, Time timeout) {
+		bulk.markUnfulfillable(clock.relativeTimeMillis());

Review comment:
        If we implement it like this, it turns to be that we require the bulk to pass every fulfill-ability check or fails otherwise. This means a request can be recognized as timed out if a slot gets lost right before its fulfill-ability check even if the bulk has passed the check last time.
   
   How about moving this `markUnfulfillable()` into the above `schedulePendingRequestBulkTimeoutCheck ()`.
    

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkChecker.java
##########
@@ -18,131 +18,14 @@
 
 package org.apache.flink.runtime.jobmaster.slotpool;
 
-import org.apache.flink.annotation.VisibleForTesting;
 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.jobmaster.SlotInfo;
-import org.apache.flink.util.clock.Clock;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Optional;
-import java.util.Set;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * This class helps to check the status of physical slot request bulks.
+ * This class tracks a fulfil-ability timeout of a bulk of physical slot requests.
+ *
+ * <p>The bulk gets canceled if the timeout occurs and the bulk is not fulfilled.

Review comment:
       `is not fulfilled` -> `is not fulfillable`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/MergingSharedSlotProfileRetrieverFactory.java
##########
@@ -41,16 +41,12 @@
 class MergingSharedSlotProfileRetrieverFactory implements SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory {
 	private final PreferredLocationsRetriever preferredLocationsRetriever;
 
-	private final Function<ExecutionVertexID, ResourceProfile> resourceProfileRetriever;
-
 	private final Function<ExecutionVertexID, AllocationID> priorAllocationIdRetriever;
 
 	MergingSharedSlotProfileRetrieverFactory(

Review comment:
       Seems extra indentation of all method params in this class are removed unexpectedly.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkCheckerImpl.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.jobmaster.slotpool;
+
+import org.apache.flink.annotation.VisibleForTesting;
+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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.util.clock.Clock;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+class PhysicalSlotRequestBulkCheckerImpl implements PhysicalSlotRequestBulkChecker {
+
+	private ComponentMainThreadExecutor componentMainThreadExecutor;
+
+	private final Supplier<Set<SlotInfo>> slotsRetriever;
+
+	private final Clock clock;
+
+	PhysicalSlotRequestBulkCheckerImpl(final Supplier<Set<SlotInfo>> slotsRetriever, final Clock clock) {
+		this.slotsRetriever = checkNotNull(slotsRetriever);
+		this.clock = checkNotNull(clock);
+	}
+
+	void start(final ComponentMainThreadExecutor mainThreadExecutor) {
+		this.componentMainThreadExecutor = mainThreadExecutor;
+	}
+
+	@Override
+	public void schedulePendingRequestBulkTimeoutCheck(PhysicalSlotRequestBulk bulk, Time timeout) {
+		schedulePendingRequestBulkTimeoutCheck(new PhysicalSlotRequestBulkWithTimestamp(bulk), timeout);
+	}
+
+	private void schedulePendingRequestBulkTimeoutCheck(PhysicalSlotRequestBulkWithTimestamp bulk, Time timeout) {
+		bulk.markUnfulfillable(clock.relativeTimeMillis());
+		componentMainThreadExecutor.schedule(() -> {
+			TimeoutCheckResult result = checkPhysicalSlotRequestBulkTimeout(bulk, timeout);
+
+			switch (result) {
+				case PENDING:
+					//re-schedule the timeout check
+					schedulePendingRequestBulkTimeoutCheck(bulk, timeout);
+					break;
+				case TIMEOUT:
+					bulk.getPhysicalSlotRequestBulk().cancel(new TimeoutException("Slot request bulk is not fulfillable!"));
+					break;
+				case FULFILLED:
+				default:
+					// no action to take
+					break;
+			}
+		}, timeout.getSize(), timeout.getUnit());
+	}
+
+	/**
+	 * Check the slot request bulk and timeout its requests if it has been unfulfillable for too long.
+	 * @param slotRequestBulk bulk of slot requests
+	 * @param slotRequestTimeout indicates how long a pending request can be unfulfillable
+	 * @return result of the check, indicating the bulk is fulfilled, still pending, or timed out
+	 */
+	@VisibleForTesting
+	TimeoutCheckResult checkPhysicalSlotRequestBulkTimeout(
+			final PhysicalSlotRequestBulkWithTimestamp slotRequestBulk,
+			final Time slotRequestTimeout) {
+
+		if (slotRequestBulk.getPhysicalSlotRequestBulk().getPendingRequests().isEmpty()) {
+			return TimeoutCheckResult.FULFILLED;
+		}
+
+		final boolean fulfillable = isSlotRequestBulkFulfillable(slotRequestBulk.getPhysicalSlotRequestBulk(), slotsRetriever);
+		if (fulfillable) {
+			slotRequestBulk.markFulfillable();
+		} else {
+			final long currentTimestamp = clock.relativeTimeMillis();
+
+			slotRequestBulk.markUnfulfillable(currentTimestamp);
+
+			final long unfulfillableSince = slotRequestBulk.getUnfulfillableSince();
+			if (unfulfillableSince + slotRequestTimeout.toMilliseconds() <= currentTimestamp) {
+				return TimeoutCheckResult.TIMEOUT;
+			}
+		}
+
+		return TimeoutCheckResult.PENDING;
+	}
+
+	/**
+	 * Returns whether the given bulk of slot requests are possible to be fulfilled at the same time
+	 * with all the reusable slots in the slot pool. A reusable slot means the slot is available or
+	 * will not be occupied indefinitely.
+	 *
+	 * @param slotRequestBulk bulk of slot requests to check
+	 * @param slotsRetriever supplies slots to be used for the fulfill-ability check
+	 * @return true if the slot requests are possible to be fulfilled, otherwise false
+	 */
+	@VisibleForTesting
+	static boolean isSlotRequestBulkFulfillable(
+			final PhysicalSlotRequestBulk slotRequestBulk,
+			final Supplier<Set<SlotInfo>> slotsRetriever) {
+
+		final Set<AllocationID> assignedSlots = slotRequestBulk.getAllocationIdsOfFulfilledRequests();
+		final Set<SlotInfo> reusableSlots = getReusableSlots(slotsRetriever, assignedSlots);
+		return areRequestsFulfillableWithSlots(slotRequestBulk.getPendingRequests(), reusableSlots);
+	}
+
+	private static Set<SlotInfo> getReusableSlots(
+			final Supplier<Set<SlotInfo>> slotsRetriever,
+			final Set<AllocationID> slotsToExclude) {
+
+		return slotsRetriever.get().stream()
+			.filter(slotInfo -> !slotInfo.willBeOccupiedIndefinitely())
+			.filter(slotInfo -> !slotsToExclude.contains(slotInfo.getAllocationId()))
+			.collect(Collectors.toSet());
+	}
+
+	private static boolean areRequestsFulfillableWithSlots(
+			final Collection<ResourceProfile> requestResourceProfiles,
+			final Set<SlotInfo> slots) {
+
+		final Set<SlotInfo> remainingSlots = new HashSet<>(slots);
+		for (ResourceProfile requestResourceProfile : requestResourceProfiles) {
+			final Optional<SlotInfo> matchedSlot = findMatchingSlotForRequest(requestResourceProfile, remainingSlots);
+			if (matchedSlot.isPresent()) {
+				remainingSlots.remove(matchedSlot.get());
+			} else {
+				return false;
+			}
+		}
+		return true;
+	}
+
+	private static Optional<SlotInfo> findMatchingSlotForRequest(
+			final ResourceProfile requestResourceProfile,
+			final Collection<SlotInfo> slots) {
+
+		return slots.stream().filter(slot -> slot.getResourceProfile().isMatching(requestResourceProfile)).findFirst();
+	}
+
+	static PhysicalSlotRequestBulkCheckerImpl fromSlotPool(final SlotPool slotPool, final Clock clock) {
+		return new PhysicalSlotRequestBulkCheckerImpl(() -> getAllSlotInfos(slotPool), clock);
+	}
+
+	private static Set<SlotInfo> getAllSlotInfos(SlotPool slotPool) {
+		return Stream
+			.concat(
+				slotPool.getAvailableSlotsInformation().stream(),
+				slotPool.getAllocatedSlotsInformation().stream())
+			.collect(Collectors.toSet());
+	}
+
+	enum TimeoutCheckResult {
+		PENDING,
+
+		FULFILLED,
+
+		TIMEOUT
+	}
+
+	@VisibleForTesting
+	static class PhysicalSlotRequestBulkWithTimestamp {

Review comment:
       How about to let it implement `PhysicalSlotRequestBulk` to simplify its usages and hide the internal `PhysicalSlotRequestBulk`? 
   And what about adding a separate class file for it? I can see we have a separate test class for it already.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SchedulerImpl.java
##########
@@ -98,14 +103,16 @@ public SchedulerImpl(
 			"Scheduler is not initialized with proper main thread executor. " +
 				"Call to Scheduler.start(...) required.");
 
-		this.bulkSlotProvider = new BulkSlotProviderImpl(slotSelectionStrategy, slotPool);
+		this.slotRequestBulkChecker = PhysicalSlotRequestBulkCheckerImpl.fromSlotPool(slotPool, SystemClock.getInstance());
+		this.bulkSlotProvider = new BulkSlotProviderImpl(slotSelectionStrategy, slotPool, slotRequestBulkChecker);
+		this.physicalSlotProvider = new PhysicalSlotProviderImpl(slotSelectionStrategy, slotPool);

Review comment:
       The `PhysicalSlotProviderImpl` ctor will invoke `slotPool.disableBatchSlotRequestTimeoutCheck()` unconditionally.
   Maybe we should move that invocation to `PhysicalSlotProviderImpl#requestNewSlot()`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkImpl.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.jobmaster.slotpool;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+
+/**
+ * Represents a bulk of physical slot requests.
+ */
+class PhysicalSlotRequestBulkImpl implements PhysicalSlotRequestBulk {

Review comment:
       I think this class can also be removed when we remove `BulkSlotProvider` and `OneSlotPerExecutionSlotAllocator`.
   But it is used in tests of `PhysicalSlotRequestBulkCheckerImpl` and `PhysicalSlotRequestBulkWithTimestamp`.
   How about adding a separate `TestingPhysicalSlotRequestBulk` for those tests? 
   I'm also fine if we do it in a separate task or when we are removing the `BulkSlotProvider`.




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



[GitHub] [flink] tillrohrmann commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r478921823



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkChecker.java
##########
@@ -18,131 +18,14 @@
 
 package org.apache.flink.runtime.jobmaster.slotpool;
 
-import org.apache.flink.annotation.VisibleForTesting;
 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.jobmaster.SlotInfo;
-import org.apache.flink.util.clock.Clock;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Optional;
-import java.util.Set;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * This class helps to check the status of physical slot request bulks.
+ * This class tracks a fulfil-ability timeout of a bulk of physical slot requests.
+ *
+ * <p>The bulk gets canceled if the timeout occurs and the bulk is not fulfillable.

Review comment:
       Yes I mean `fulfil-ability`. I would suggest to either write `fulfil` or `fulfillability`.




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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202",
       "triggerID" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219",
       "triggerID" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6299",
       "triggerID" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6307",
       "triggerID" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "23cbf376a6cc07924cf9101a95962a34359c1b76",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "23cbf376a6cc07924cf9101a95962a34359c1b76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8dde2eb2e2bb41c213d23f874be8392365f2c4f9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6328",
       "triggerID" : "8dde2eb2e2bb41c213d23f874be8392365f2c4f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * d01d63a8d22b82b6b8168d3abed0505191d655e6 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6307) 
   * 23cbf376a6cc07924cf9101a95962a34359c1b76 UNKNOWN
   * 8dde2eb2e2bb41c213d23f874be8392365f2c4f9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6328) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * 18f88af3b438b13e9a240efd2b4979f841d2b978 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939) 
   * a96b2db52a0db507e0077266c8e9cb947413e1ba Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 347df14dd8621cf7922f723aa5faf96a4f4fc1c9 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671) 
   * 7e07e70b62850e70644e35b18c9168426371862c UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r478552732



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkChecker.java
##########
@@ -18,131 +18,14 @@
 
 package org.apache.flink.runtime.jobmaster.slotpool;
 
-import org.apache.flink.annotation.VisibleForTesting;
 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.jobmaster.SlotInfo;
-import org.apache.flink.util.clock.Clock;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Optional;
-import java.util.Set;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * This class helps to check the status of physical slot request bulks.
+ * This class tracks a fulfil-ability timeout of a bulk of physical slot requests.
+ *
+ * <p>The bulk gets canceled if the timeout occurs and the bulk is not fulfillable.

Review comment:
       Do you mean `fulfil-ability`? I was not sure how to write it as the word itself does not seem to exist but it is quite expressive in our case, imo.




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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9e7d2cb792556814b91b8f580d9a1cbe98433274 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820) 
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * bce753ff7f4da2cffa295bd4007517af4c5697d8 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] zhuzhurk commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r484093835



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -161,158 +173,66 @@ private SharedSlot getOrAllocateSharedSlot(
 					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
 						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
 					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
-				return new SharedSlot(physicalSlotRequestId, physicalSlotResourceProfile, group, physicalSlotFuture);
+				return new SharedSlot(
+					physicalSlotRequestId,
+					physicalSlotResourceProfile,
+					group,
+					physicalSlotFuture,
+					slotWillBeOccupiedIndefinitely,
+					this::releaseSharedSlot);
 			});
 	}
 
+	private void releaseSharedSlot(ExecutionSlotSharingGroup executionSlotSharingGroup) {
+		SharedSlot slot = sharedSlots.remove(executionSlotSharingGroup);
+		if (slot != null) {
+			slotProvider.cancelSlotRequest(
+				slot.getPhysicalSlotRequestId(),
+				new FlinkException("Slot is being returned from SlotSharingExecutionSlotAllocator."));
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup {} to release", executionSlotSharingGroup);

Review comment:
       I just found a case that this branch can happen and the latest `checkNotNull(slot)` will fail.
   The case is that a task finishes and returns its logical slot which is the only logical slot of a `SharedSlot`. `SharedSlot#returnLogicalSlot()` will be invoked first to release the shared slot and underlying physical slot. The releasing of physical slot, however, could trigger `SharedSlot#release()` which will try to release this shared slot once more.
   I think we need either to add RELEASED/ACTIVE states for `SharedSlot` so that it can skip duplicated releasing. Or we need to keep the `if (slot != null)` condition block here. Anyway, the `else` block does not make much sense so I think we can drop it.
   WDYT?




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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r475234042



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java
##########
@@ -274,6 +277,73 @@ public void testPhysicalSlotReleaseLogicalSlots() throws ExecutionException, Int
 		assertThat(payloads.stream().allMatch(payload -> payload.getTerminalStateFuture().isDone()), is(true));
 	}
 
+	@Test
+	public void testSchedulePendingRequestBulkTimeoutCheck() {
+		TestingPhysicalSlotRequestBulkChecker bulkChecker = new TestingPhysicalSlotRequestBulkChecker();
+		AllocationContext context = createBulkCheckerContextWithEv12GroupAndEv3Group(bulkChecker);
+
+		context.allocateSlotsFor(EV1, EV3);
+		PhysicalSlotRequestBulk bulk = bulkChecker.getBulk();
+
+		assertThat(bulk.getPendingRequests(), hasSize(2));
+		assertThat(bulk.getPendingRequests(), containsInAnyOrder(RESOURCE_PROFILE.multiply(2), RESOURCE_PROFILE));
+		assertThat(bulk.getAllocationIdsOfFulfilledRequests(), hasSize(0));
+		assertThat(bulkChecker.getTimeout(), is(ALLOCATION_TIMEOUT));
+	}
+
+	@Test
+	public void testRequestFulfilledInBulk() {
+		TestingPhysicalSlotRequestBulkChecker bulkChecker = new TestingPhysicalSlotRequestBulkChecker();
+		AllocationContext context = createBulkCheckerContextWithEv12GroupAndEv3Group(bulkChecker);
+
+		context.allocateSlotsFor(EV1, EV3);
+		AllocationID allocationId = new AllocationID();
+		ResourceProfile pendingSlotResourceProfile = fulfilOneOfTwoSlotRequestsAndGetPendingProfile(context, allocationId);
+		PhysicalSlotRequestBulk bulk = bulkChecker.getBulk();
+
+		assertThat(bulk.getPendingRequests(), hasSize(1));
+		assertThat(bulk.getPendingRequests(), containsInAnyOrder(pendingSlotResourceProfile));
+		assertThat(bulk.getAllocationIdsOfFulfilledRequests(), hasSize(1));
+		assertThat(bulk.getAllocationIdsOfFulfilledRequests(), containsInAnyOrder(allocationId));
+	}
+
+	@Test
+	public void testRequestBulkCancel() {
+		TestingPhysicalSlotRequestBulkChecker bulkChecker = new TestingPhysicalSlotRequestBulkChecker();
+		AllocationContext context = createBulkCheckerContextWithEv12GroupAndEv3Group(bulkChecker);
+
+		// allocate 2 physical slots for 2 groups
+		List<SlotExecutionVertexAssignment> assignments1 = context.allocateSlotsFor(EV1, EV3);
+		fulfilOneOfTwoSlotRequestsAndGetPendingProfile(context, new AllocationID());
+		PhysicalSlotRequestBulk bulk1 = bulkChecker.getBulk();
+		List<SlotExecutionVertexAssignment> assignments2 = context.allocateSlotsFor(EV2);
+		// cancelling of (EV1, EV3) releases assignments1 and only one physical slot for EV3
+		// the second physical slot is held by sharing EV2 from the next bulk
+		bulk1.cancel(new Throwable());
+		// EV3 needs again a physical slot, therefore there are 3 requests overall
+		context.allocateSlotsFor(EV1, EV3);
+		boolean ev1failed = assignments1.get(0).getLogicalSlotFuture().isCompletedExceptionally();
+		boolean ev2failed = assignments1.get(0).getLogicalSlotFuture().isCompletedExceptionally();
+
+		assertThat(context.getSlotProvider().getRequests().values(), hasSize(3));
+		// either EV1 or EV3 logical slot future is fulfilled before cancellation
+		assertThat(ev1failed != ev2failed, is(false));
+		assertThat(assignments2.get(0).getLogicalSlotFuture().isCompletedExceptionally(), is(false));

Review comment:
       this is not necessarily true because `fulfilOneOfTwoSlotRequestsAndGetPendingProfile` fulfils only one physical slot request, either for (EV1, EV2) or (EV3). If it fulfils the physical slot request for (EV1, EV2), (EV3) is still pending. The important bit was that one of [EV1, EV3] bulk gets canceled if the [EV1, EV3] bulk is cancelled but not EV3 as it is from another bulk.




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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202",
       "triggerID" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219",
       "triggerID" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6299",
       "triggerID" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * db1607ead33ae1b60634c1162ba8247a9957f0f4 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219) 
   * 7b527d1937d410eaa7de59479915074ca02e6fbd Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6299) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9e7d2cb792556814b91b8f580d9a1cbe98433274 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820) 
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * bce753ff7f4da2cffa295bd4007517af4c5697d8 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202",
       "triggerID" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219",
       "triggerID" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6299",
       "triggerID" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6307",
       "triggerID" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "23cbf376a6cc07924cf9101a95962a34359c1b76",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "23cbf376a6cc07924cf9101a95962a34359c1b76",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * d01d63a8d22b82b6b8168d3abed0505191d655e6 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6307) 
   * 23cbf376a6cc07924cf9101a95962a34359c1b76 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * bce753ff7f4da2cffa295bd4007517af4c5697d8 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926) 
   * 18f88af3b438b13e9a240efd2b4979f841d2b978 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] zhuzhurk commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r483462252



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java
##########
@@ -95,12 +97,15 @@
 
 	private final Set<ExecutionVertexID> verticesWaitingForRestart;
 
+	private final Consumer<ComponentMainThreadExecutor> slartUpAction;

Review comment:
       `slartUpAction` -> `startUpAction `

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotProvider.java
##########
@@ -62,11 +51,13 @@ default void start(ComponentMainThreadExecutor mainThreadExecutor) {
 	 * @param allocationTimeout after which the allocation fails with a timeout exception
 	 * @return The future of the allocation
 	 */
-	CompletableFuture<LogicalSlot> allocateSlot(
+	default CompletableFuture<LogicalSlot> allocateSlot(
 		SlotRequestId slotRequestId,
 		ScheduledUnit scheduledUnit,
 		SlotProfile slotProfile,
-		Time allocationTimeout);
+		Time allocationTimeout) {

Review comment:
       Looks to me default body is not needed. I tried removing the default body and flink-runtime still compiles.
   Correct me if I miss anything.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java
##########
@@ -125,24 +153,50 @@ static SchedulingStrategyFactory createSchedulingStrategyFactory(final ScheduleM
 		}
 	}
 
-	private static ExecutionSlotAllocatorFactory createExecutionSlotAllocatorFactory(
+	private static DefaultSchedulerComponents createPipelinedRegionSchedulerComponents(
 			final ScheduleMode scheduleMode,
-			final SlotProvider slotProvider,
-			final Time slotRequestTimeout,
-			final SchedulingStrategyFactory schedulingStrategyFactory) {
-
-		if (schedulingStrategyFactory instanceof PipelinedRegionSchedulingStrategy.Factory) {
-			return new OneSlotPerExecutionSlotAllocatorFactory(
-				slotProvider,
-				scheduleMode != ScheduleMode.LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST,
-				slotRequestTimeout);
-		} else {
-			final SlotProviderStrategy slotProviderStrategy = SlotProviderStrategy.from(
-				scheduleMode,
-				slotProvider,
-				slotRequestTimeout);
-
-			return new DefaultExecutionSlotAllocatorFactory(slotProviderStrategy);
+			final Configuration jobMasterConfiguration,
+			final SlotPool slotPool,
+			final Time slotRequestTimeout) {
+		final SlotSelectionStrategy slotSelectionStrategy = selectSlotSelectionStrategy(jobMasterConfiguration);
+		final BulkSlotProvider bulkSlotProvider = new BulkSlotProviderImpl(slotSelectionStrategy, slotPool);
+		final ExecutionSlotAllocatorFactory allocatorFactory = new OneSlotPerExecutionSlotAllocatorFactory(
+			bulkSlotProvider,
+			scheduleMode != ScheduleMode.LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST,
+			slotRequestTimeout);
+		return new DefaultSchedulerComponents(
+			new PipelinedRegionSchedulingStrategy.Factory(),
+			bulkSlotProvider::start,
+			allocatorFactory);
+	}
+
+	@Nonnull
+	private static SlotSelectionStrategy selectSlotSelectionStrategy(@Nonnull Configuration configuration) {
+		final boolean evenlySpreadOutSlots = configuration.getBoolean(ClusterOptions.EVENLY_SPREAD_OUT_SLOTS_STRATEGY);
+
+		final SlotSelectionStrategy locationPreferenceSlotSelectionStrategy;
+
+		locationPreferenceSlotSelectionStrategy = evenlySpreadOutSlots ?
+			LocationPreferenceSlotSelectionStrategy.createEvenlySpreadOut() :
+			LocationPreferenceSlotSelectionStrategy.createDefault();
+
+		return configuration.getBoolean(CheckpointingOptions.LOCAL_RECOVERY) ?
+			PreviousAllocationSlotSelectionStrategy.create(locationPreferenceSlotSelectionStrategy) :
+			locationPreferenceSlotSelectionStrategy;
+	}
+
+	private static class DefaultSchedulerComponents {
+		private final SchedulingStrategyFactory schedulingStrategyFactory;
+		private final Consumer<ComponentMainThreadExecutor> slartUpAction;

Review comment:
       `startUpAction` -> `startUpAction`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java
##########
@@ -125,24 +153,50 @@ static SchedulingStrategyFactory createSchedulingStrategyFactory(final ScheduleM
 		}
 	}
 
-	private static ExecutionSlotAllocatorFactory createExecutionSlotAllocatorFactory(
+	private static DefaultSchedulerComponents createPipelinedRegionSchedulerComponents(
 			final ScheduleMode scheduleMode,
-			final SlotProvider slotProvider,
-			final Time slotRequestTimeout,
-			final SchedulingStrategyFactory schedulingStrategyFactory) {
-
-		if (schedulingStrategyFactory instanceof PipelinedRegionSchedulingStrategy.Factory) {
-			return new OneSlotPerExecutionSlotAllocatorFactory(
-				slotProvider,
-				scheduleMode != ScheduleMode.LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST,
-				slotRequestTimeout);
-		} else {
-			final SlotProviderStrategy slotProviderStrategy = SlotProviderStrategy.from(
-				scheduleMode,
-				slotProvider,
-				slotRequestTimeout);
-
-			return new DefaultExecutionSlotAllocatorFactory(slotProviderStrategy);
+			final Configuration jobMasterConfiguration,
+			final SlotPool slotPool,
+			final Time slotRequestTimeout) {
+		final SlotSelectionStrategy slotSelectionStrategy = selectSlotSelectionStrategy(jobMasterConfiguration);
+		final BulkSlotProvider bulkSlotProvider = new BulkSlotProviderImpl(slotSelectionStrategy, slotPool);
+		final ExecutionSlotAllocatorFactory allocatorFactory = new OneSlotPerExecutionSlotAllocatorFactory(
+			bulkSlotProvider,
+			scheduleMode != ScheduleMode.LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST,
+			slotRequestTimeout);
+		return new DefaultSchedulerComponents(
+			new PipelinedRegionSchedulingStrategy.Factory(),
+			bulkSlotProvider::start,
+			allocatorFactory);
+	}
+
+	@Nonnull

Review comment:
       We can remove the `@Nonnull` annotation I think.
   This is because that we will by default assume it to be non-null, 
   and only `@Nullable` annotations are required when needed.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotProvider.java
##########
@@ -102,31 +93,19 @@ default void start(ComponentMainThreadExecutor mainThreadExecutor) {
 			allocationTimeout);
 	}
 
-	/**
-	 * Allocates a bulk of physical slots. The allocation will be completed
-	 * normally only when all the requests are fulfilled.
-	 *
-	 * @param physicalSlotRequests requests for physical slots
-	 * @param timeout indicating how long it is accepted that the slot requests can be unfulfillable
-	 * @return future of the results of slot requests
-	 */
-	default CompletableFuture<Collection<PhysicalSlotRequest.Result>> allocatePhysicalSlots(
-		Collection<PhysicalSlotRequest> physicalSlotRequests,
-		Time timeout) {
-		throw new UnsupportedOperationException("Not properly implemented.");
-	}
-
 	/**
 	 * Cancels the slot request with the given {@link SlotRequestId} and {@link SlotSharingGroupId}.
 	 *
 	 * @param slotRequestId identifying the slot request to cancel
 	 * @param slotSharingGroupId identifying the slot request to cancel
 	 * @param cause of the cancellation
 	 */
-	void cancelSlotRequest(
-		SlotRequestId slotRequestId,
-		@Nullable SlotSharingGroupId slotSharingGroupId,
-		Throwable cause);
+	default void cancelSlotRequest(

Review comment:
       Looks to me default body is not needed. I tried removing the default body and flink-runtime still compiles.
   Correct me if I miss anything.




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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r484542448



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -161,158 +173,66 @@ private SharedSlot getOrAllocateSharedSlot(
 					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
 						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
 					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
-				return new SharedSlot(physicalSlotRequestId, physicalSlotResourceProfile, group, physicalSlotFuture);
+				return new SharedSlot(
+					physicalSlotRequestId,
+					physicalSlotResourceProfile,
+					group,
+					physicalSlotFuture,
+					slotWillBeOccupiedIndefinitely,
+					this::releaseSharedSlot);
 			});
 	}
 
+	private void releaseSharedSlot(ExecutionSlotSharingGroup executionSlotSharingGroup) {
+		SharedSlot slot = sharedSlots.remove(executionSlotSharingGroup);
+		if (slot != null) {
+			slotProvider.cancelSlotRequest(
+				slot.getPhysicalSlotRequestId(),
+				new FlinkException("Slot is being returned from SlotSharingExecutionSlotAllocator."));
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup {} to release", executionSlotSharingGroup);

Review comment:
       alright, I added ALLOCATED/RELEASED states and guards for them




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



[GitHub] [flink] azagrebin closed pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin closed pull request #13181:
URL: https://github.com/apache/flink/pull/13181


   


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



[GitHub] [flink] zhuzhurk commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r475330349



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkImpl.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.jobmaster.slotpool;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+
+/**
+ * Represents a bulk of physical slot requests.
+ */
+class PhysicalSlotRequestBulkImpl implements PhysicalSlotRequestBulk {

Review comment:
       Sounds good to me.




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



[GitHub] [flink] zhuzhurk commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r474639086



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java
##########
@@ -274,6 +277,73 @@ public void testPhysicalSlotReleaseLogicalSlots() throws ExecutionException, Int
 		assertThat(payloads.stream().allMatch(payload -> payload.getTerminalStateFuture().isDone()), is(true));
 	}
 
+	@Test
+	public void testSchedulePendingRequestBulkTimeoutCheck() {
+		TestingPhysicalSlotRequestBulkChecker bulkChecker = new TestingPhysicalSlotRequestBulkChecker();
+		AllocationContext context = createBulkCheckerContextWithEv12GroupAndEv3Group(bulkChecker);
+
+		context.allocateSlotsFor(EV1, EV3);
+		PhysicalSlotRequestBulk bulk = bulkChecker.getBulk();
+
+		assertThat(bulk.getPendingRequests(), hasSize(2));
+		assertThat(bulk.getPendingRequests(), containsInAnyOrder(RESOURCE_PROFILE.multiply(2), RESOURCE_PROFILE));
+		assertThat(bulk.getAllocationIdsOfFulfilledRequests(), hasSize(0));
+		assertThat(bulkChecker.getTimeout(), is(ALLOCATION_TIMEOUT));
+	}
+
+	@Test
+	public void testRequestFulfilledInBulk() {
+		TestingPhysicalSlotRequestBulkChecker bulkChecker = new TestingPhysicalSlotRequestBulkChecker();
+		AllocationContext context = createBulkCheckerContextWithEv12GroupAndEv3Group(bulkChecker);
+
+		context.allocateSlotsFor(EV1, EV3);
+		AllocationID allocationId = new AllocationID();
+		ResourceProfile pendingSlotResourceProfile = fulfilOneOfTwoSlotRequestsAndGetPendingProfile(context, allocationId);
+		PhysicalSlotRequestBulk bulk = bulkChecker.getBulk();
+
+		assertThat(bulk.getPendingRequests(), hasSize(1));
+		assertThat(bulk.getPendingRequests(), containsInAnyOrder(pendingSlotResourceProfile));
+		assertThat(bulk.getAllocationIdsOfFulfilledRequests(), hasSize(1));
+		assertThat(bulk.getAllocationIdsOfFulfilledRequests(), containsInAnyOrder(allocationId));
+	}
+
+	@Test
+	public void testRequestBulkCancel() {
+		TestingPhysicalSlotRequestBulkChecker bulkChecker = new TestingPhysicalSlotRequestBulkChecker();
+		AllocationContext context = createBulkCheckerContextWithEv12GroupAndEv3Group(bulkChecker);
+
+		// allocate 2 physical slots for 2 groups
+		List<SlotExecutionVertexAssignment> assignments1 = context.allocateSlotsFor(EV1, EV3);
+		fulfilOneOfTwoSlotRequestsAndGetPendingProfile(context, new AllocationID());
+		PhysicalSlotRequestBulk bulk1 = bulkChecker.getBulk();
+		List<SlotExecutionVertexAssignment> assignments2 = context.allocateSlotsFor(EV2);
+		// cancelling of (EV1, EV3) releases assignments1 and only one physical slot for EV3
+		// the second physical slot is held by sharing EV2 from the next bulk
+		bulk1.cancel(new Throwable());
+		// EV3 needs again a physical slot, therefore there are 3 requests overall
+		context.allocateSlotsFor(EV1, EV3);
+		boolean ev1failed = assignments1.get(0).getLogicalSlotFuture().isCompletedExceptionally();
+		boolean ev2failed = assignments1.get(0).getLogicalSlotFuture().isCompletedExceptionally();
+
+		assertThat(context.getSlotProvider().getRequests().values(), hasSize(3));
+		// either EV1 or EV3 logical slot future is fulfilled before cancellation
+		assertThat(ev1failed != ev2failed, is(false));

Review comment:
       do you actually want to verify `assertThat(ev1failed != ev2failed, is(true));` ?




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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r483450043



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SchedulerImpl.java
##########
@@ -98,14 +101,15 @@ public SchedulerImpl(
 			"Scheduler is not initialized with proper main thread executor. " +
 				"Call to Scheduler.start(...) required.");
 
-		this.bulkSlotProvider = new BulkSlotProviderImpl(slotSelectionStrategy, slotPool);
+		this.slotRequestBulkChecker = PhysicalSlotRequestBulkCheckerImpl.fromSlotPool(slotPool, SystemClock.getInstance());
+		this.bulkSlotProvider = new BulkSlotProviderImpl(slotSelectionStrategy, slotPool, slotRequestBulkChecker);

Review comment:
       Alright, thanks for the idea. It required a bit more changes but it looks working. The change is the commit:
   `[hotfix] Do not mix BulkSlotProvider into SlotProvider/SchedulerImpl`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotProviderImpl.java
##########
@@ -87,9 +86,12 @@
 			SlotRequestId slotRequestId,
 			ResourceProfile resourceProfile,
 			boolean willSlotBeOccupiedIndefinitely) {
-		return willSlotBeOccupiedIndefinitely ?
-			slotPool.requestNewAllocatedSlot(slotRequestId, resourceProfile, null) :
-			slotPool.requestNewAllocatedBatchSlot(slotRequestId, resourceProfile);
+		if (willSlotBeOccupiedIndefinitely) {
+			return slotPool.requestNewAllocatedSlot(slotRequestId, resourceProfile, null);
+		} else {
+			slotPool.disableBatchSlotRequestTimeoutCheck();

Review comment:
       true




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



[GitHub] [flink] azagrebin commented on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-678790798


   Thanks for the review @zhuzhurk 
   I addressed the comments


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



[GitHub] [flink] azagrebin commented on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-687108594


   Thanks for the review @tillrohrmann and @zhuzhurk 
   I have addressed comments


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * 18f88af3b438b13e9a240efd2b4979f841d2b978 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] zhuzhurk commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r484093835



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -161,158 +173,66 @@ private SharedSlot getOrAllocateSharedSlot(
 					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
 						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
 					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
-				return new SharedSlot(physicalSlotRequestId, physicalSlotResourceProfile, group, physicalSlotFuture);
+				return new SharedSlot(
+					physicalSlotRequestId,
+					physicalSlotResourceProfile,
+					group,
+					physicalSlotFuture,
+					slotWillBeOccupiedIndefinitely,
+					this::releaseSharedSlot);
 			});
 	}
 
+	private void releaseSharedSlot(ExecutionSlotSharingGroup executionSlotSharingGroup) {
+		SharedSlot slot = sharedSlots.remove(executionSlotSharingGroup);
+		if (slot != null) {
+			slotProvider.cancelSlotRequest(
+				slot.getPhysicalSlotRequestId(),
+				new FlinkException("Slot is being returned from SlotSharingExecutionSlotAllocator."));
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup {} to release", executionSlotSharingGroup);

Review comment:
       I just found a case that this branch can happen and the latest `checkNotNull(slot)` will fail.
   The case is that a task finishes and returns its logical slot which is the only logical slot of a `SharedSlot`. `SharedSlot#returnLogicalSlot()` will be invoked first to release the shared slot and underlying physical slot. The releasing of physical slot, however, would release its payload and trigger `SharedSlot#release()` which will try to release this shared slot once more.
   I think we need either to add RELEASED/ACTIVE states for `SharedSlot` so that it can skip duplicated releasing. Or we need to keep the `if (slot != null)` condition block here. Anyway, the `else` block does not make much sense so I think we can drop it.
   WDYT?




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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r484711637



##########
File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackendBuilder.java
##########
@@ -181,7 +181,7 @@ public RocksDBKeyedStateBackendBuilder(
 		RocksDB injectedTestDB,
 		ColumnFamilyHandle injectedDefaultColumnFamilyHandle,
 		CloseableRegistry cancelStreamRegistry) {
-		this(
+		 this(

Review comment:
       true :) reverted




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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r481855755



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -161,158 +173,66 @@ private SharedSlot getOrAllocateSharedSlot(
 					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
 						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
 					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
-				return new SharedSlot(physicalSlotRequestId, physicalSlotResourceProfile, group, physicalSlotFuture);
+				return new SharedSlot(
+					physicalSlotRequestId,
+					physicalSlotResourceProfile,
+					group,
+					physicalSlotFuture,
+					slotWillBeOccupiedIndefinitely,
+					this::releaseSharedSlot);
 			});
 	}
 
+	private void releaseSharedSlot(ExecutionSlotSharingGroup executionSlotSharingGroup) {
+		SharedSlot slot = sharedSlots.remove(executionSlotSharingGroup);
+		if (slot != null) {
+			slotProvider.cancelSlotRequest(
+				slot.getPhysicalSlotRequestId(),
+				new FlinkException("Slot is being returned from SlotSharingExecutionSlotAllocator."));
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup {} to release", executionSlotSharingGroup);

Review comment:
       True, I think it is some leftover




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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202",
       "triggerID" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219",
       "triggerID" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6299",
       "triggerID" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6307",
       "triggerID" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "23cbf376a6cc07924cf9101a95962a34359c1b76",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "23cbf376a6cc07924cf9101a95962a34359c1b76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8dde2eb2e2bb41c213d23f874be8392365f2c4f9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8dde2eb2e2bb41c213d23f874be8392365f2c4f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * d01d63a8d22b82b6b8168d3abed0505191d655e6 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6307) 
   * 23cbf376a6cc07924cf9101a95962a34359c1b76 UNKNOWN
   * 8dde2eb2e2bb41c213d23f874be8392365f2c4f9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * bce753ff7f4da2cffa295bd4007517af4c5697d8 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r475569737



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkCheckerImplTest.java
##########
@@ -0,0 +1,257 @@
+/*
+ * 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.jobmaster.slotpool;
+
+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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.clock.ManualClock;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+import static org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotTestUtils.createPhysicalSlot;
+import static org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotTestUtils.occupyPhysicalSlot;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for {@link PhysicalSlotRequestBulkCheckerImpl}.
+ */
+public class PhysicalSlotRequestBulkCheckerImplTest extends TestLogger {
+
+	private static final Time TIMEOUT = Time.milliseconds(100L);
+
+	private static ScheduledExecutorService singleThreadScheduledExecutorService;
+
+	private static ComponentMainThreadExecutor mainThreadExecutor;
+
+	private final ManualClock clock = new ManualClock();
+
+	private PhysicalSlotRequestBulkCheckerImpl bulkChecker;
+
+	private Set<PhysicalSlot> slots;
+
+	private Supplier<Set<SlotInfo>> slotsRetriever;
+
+	@BeforeClass
+	public static void setupClass() {
+		singleThreadScheduledExecutorService = Executors.newSingleThreadScheduledExecutor();
+		mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forSingleThreadExecutor(singleThreadScheduledExecutorService);
+	}
+
+	@AfterClass
+	public static void teardownClass() {
+		if (singleThreadScheduledExecutorService != null) {
+			singleThreadScheduledExecutorService.shutdownNow();
+		}
+	}
+
+	@Before
+	public void setup() throws Exception {
+		slots = new HashSet<>();
+		slotsRetriever = () -> new HashSet<>(slots);
+		bulkChecker = new PhysicalSlotRequestBulkCheckerImpl(slotsRetriever, clock);
+		bulkChecker.start(mainThreadExecutor);
+	}
+
+	@Test
+	public void testPendingBulkIsNotCancelled() throws InterruptedException, ExecutionException {
+		final CompletableFuture<SlotRequestId> cancellationFuture = new CompletableFuture<>();
+		final PhysicalSlotRequestBulk bulk = createPhysicalSlotRequestBulkWithCancellationFuture(cancellationFuture, new SlotRequestId());
+		bulkChecker.schedulePendingRequestBulkTimeoutCheck(bulk, TIMEOUT);
+		checkNotCancelledAfter(cancellationFuture, 2 * TIMEOUT.toMilliseconds());
+	}
+
+	@Test
+	public void testFulfilledBulkIsNotCancelled() throws InterruptedException, ExecutionException {
+		final CompletableFuture<SlotRequestId> cancellationFuture = new CompletableFuture<>();
+		final PhysicalSlotRequestBulk bulk = createPhysicalSlotRequestBulkWithCancellationFuture(cancellationFuture, new SlotRequestId());
+		bulkChecker.schedulePendingRequestBulkTimeoutCheck(bulk, TIMEOUT);
+		checkNotCancelledAfter(cancellationFuture, 2  * TIMEOUT.toMilliseconds());
+	}
+
+	private static void checkNotCancelledAfter(
+			CompletableFuture<?> cancellationFuture, long milli) throws ExecutionException, InterruptedException {
+		mainThreadExecutor.schedule(() -> {}, milli, TimeUnit.MILLISECONDS).get();
+		try {
+			cancellationFuture.get(milli, TimeUnit.MILLISECONDS);

Review comment:
       True, I can add it. Although we have to wait in usual case.




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



[GitHub] [flink] flinkbot commented on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b54bb16026ebc704e7dbab97bba1f18e0438264b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r481293405



##########
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());

Review comment:
       it does not accept collection which is what `fulfilledRequests.values()` is




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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202",
       "triggerID" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219",
       "triggerID" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6299",
       "triggerID" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6307",
       "triggerID" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "23cbf376a6cc07924cf9101a95962a34359c1b76",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "23cbf376a6cc07924cf9101a95962a34359c1b76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8dde2eb2e2bb41c213d23f874be8392365f2c4f9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6328",
       "triggerID" : "8dde2eb2e2bb41c213d23f874be8392365f2c4f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fe8cc1fc15e29ede254333ae186eb0b6cb62d5a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9fe8cc1fc15e29ede254333ae186eb0b6cb62d5a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dbed0f154c63a16b28530d8d1a5ff351ea64f3d0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6331",
       "triggerID" : "dbed0f154c63a16b28530d8d1a5ff351ea64f3d0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * d01d63a8d22b82b6b8168d3abed0505191d655e6 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6307) 
   * 23cbf376a6cc07924cf9101a95962a34359c1b76 UNKNOWN
   * 8dde2eb2e2bb41c213d23f874be8392365f2c4f9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6328) 
   * 9fe8cc1fc15e29ede254333ae186eb0b6cb62d5a UNKNOWN
   * dbed0f154c63a16b28530d8d1a5ff351ea64f3d0 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6331) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 347df14dd8621cf7922f723aa5faf96a4f4fc1c9 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] tillrohrmann commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r479998734



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SchedulerImpl.java
##########
@@ -98,14 +101,15 @@ public SchedulerImpl(
 			"Scheduler is not initialized with proper main thread executor. " +
 				"Call to Scheduler.start(...) required.");
 
-		this.bulkSlotProvider = new BulkSlotProviderImpl(slotSelectionStrategy, slotPool);
+		this.slotRequestBulkChecker = PhysicalSlotRequestBulkCheckerImpl.fromSlotPool(slotPool, SystemClock.getInstance());
+		this.bulkSlotProvider = new BulkSlotProviderImpl(slotSelectionStrategy, slotPool, slotRequestBulkChecker);

Review comment:
       It might make sense to tackle these kind of things in the beginning and not at the end for future developments.




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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202",
       "triggerID" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219",
       "triggerID" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * db1607ead33ae1b60634c1162ba8247a9957f0f4 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219) 
   * 7b527d1937d410eaa7de59479915074ca02e6fbd UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r475110865



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkImpl.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.jobmaster.slotpool;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+
+/**
+ * Represents a bulk of physical slot requests.
+ */
+class PhysicalSlotRequestBulkImpl implements PhysicalSlotRequestBulk {

Review comment:
       I agree, this is a good point. I do not have strong opinion when to do it. However, I would prefer to move `PhysicalSlotRequestBulkImpl` to tests during the removal of `OneSlotPerExecution` code as it is quite simple and we will not have duplications meanwhile. wdyt?




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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202",
       "triggerID" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219",
       "triggerID" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6299",
       "triggerID" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6307",
       "triggerID" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "23cbf376a6cc07924cf9101a95962a34359c1b76",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "23cbf376a6cc07924cf9101a95962a34359c1b76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8dde2eb2e2bb41c213d23f874be8392365f2c4f9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6328",
       "triggerID" : "8dde2eb2e2bb41c213d23f874be8392365f2c4f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fe8cc1fc15e29ede254333ae186eb0b6cb62d5a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9fe8cc1fc15e29ede254333ae186eb0b6cb62d5a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * d01d63a8d22b82b6b8168d3abed0505191d655e6 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6307) 
   * 23cbf376a6cc07924cf9101a95962a34359c1b76 UNKNOWN
   * 8dde2eb2e2bb41c213d23f874be8392365f2c4f9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6328) 
   * 9fe8cc1fc15e29ede254333ae186eb0b6cb62d5a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202",
       "triggerID" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219",
       "triggerID" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * d34648bab5e9a642c97db6e98f80f8059500c145 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202) 
   * db1607ead33ae1b60634c1162ba8247a9957f0f4 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b54bb16026ebc704e7dbab97bba1f18e0438264b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r475113727



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkCheckerImplTest.java
##########
@@ -0,0 +1,274 @@
+/*
+ * 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.jobmaster.slotpool;
+
+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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequestBulkCheckerImpl.PhysicalSlotRequestBulkWithTimestamp;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.clock.ManualClock;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+import static org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotTestUtils.createPhysicalSlot;
+import static org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotTestUtils.occupyPhysicalSlot;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for {@link PhysicalSlotRequestBulkCheckerImpl}.
+ */
+public class PhysicalSlotRequestBulkCheckerImplTest extends TestLogger {
+
+	private static final Time TIMEOUT = Time.milliseconds(100L);
+
+	private static ScheduledExecutorService singleThreadScheduledExecutorService;
+
+	private static ComponentMainThreadExecutor mainThreadExecutor;
+
+	private final ManualClock clock = new ManualClock();
+
+	private PhysicalSlotRequestBulkCheckerImpl bulkChecker;
+
+	private Set<PhysicalSlot> slots;
+
+	private Supplier<Set<SlotInfo>> slotsRetriever;
+
+	@BeforeClass
+	public static void setupClass() {
+		singleThreadScheduledExecutorService = Executors.newSingleThreadScheduledExecutor();
+		mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forSingleThreadExecutor(singleThreadScheduledExecutorService);
+	}
+
+	@AfterClass
+	public static void teardownClass() {
+		if (singleThreadScheduledExecutorService != null) {
+			singleThreadScheduledExecutorService.shutdownNow();
+		}
+	}
+
+	@Before
+	public void setup() throws Exception {
+		slots = new HashSet<>();
+		slotsRetriever = () -> new HashSet<>(slots);
+		bulkChecker = new PhysicalSlotRequestBulkCheckerImpl(slotsRetriever, clock);
+		bulkChecker.start(mainThreadExecutor);
+	}
+
+	@Test
+	public void testPendingBulkIsNotCancelled() throws InterruptedException, ExecutionException {
+		final PhysicalSlotRequestBulkWithCancellationTracking bulk = new PhysicalSlotRequestBulkWithCancellationTracking(ResourceProfile.ANY);
+		bulkChecker.schedulePendingRequestBulkTimeoutCheck(bulk, TIMEOUT);
+		bulk.checkNotCancelledAfter(TIMEOUT.toMilliseconds() + 10);

Review comment:
       True, the problem is also we should not use long waits in tests as it increases the overall running time.
   How about we schedule something and join on it after `schedulePendingRequestBulkTimeoutCheck`?
   `mainThreadExecutor.schedule(() -> {}, 2*TIMEOUT.toMilliseconds(), TimeUnit.MILLISECONDS).get();`
   the check should be executed before the dummy runnable. wdyt?




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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7e07e70b62850e70644e35b18c9168426371862c Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794) 
   * 9e7d2cb792556814b91b8f580d9a1cbe98433274 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9e7d2cb792556814b91b8f580d9a1cbe98433274 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820) 
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r481844415



##########
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));
+	}
+
+	void markFulfilled(ExecutionSlotSharingGroup group, AllocationID allocationID) {
+		pendingRequests.remove(group);
+		fulfilledRequests.put(group, allocationID);
+	}
+
+	void clear() {
+		pendingRequests.clear();

Review comment:
       It is written where it is used atm.
   The class specifically exists for the `SlotSharingExecutionSlotAllocator`.
   I can rename the method to `clearPendingRequests`.




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



[GitHub] [flink] zhuzhurk commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r484708915



##########
File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackendBuilder.java
##########
@@ -181,7 +181,7 @@ public RocksDBKeyedStateBackendBuilder(
 		RocksDB injectedTestDB,
 		ColumnFamilyHandle injectedDefaultColumnFamilyHandle,
 		CloseableRegistry cancelStreamRegistry) {
-		this(
+		 this(

Review comment:
       Unintentional change I guess?




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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * 18f88af3b438b13e9a240efd2b4979f841d2b978 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939) 
   * a96b2db52a0db507e0077266c8e9cb947413e1ba UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r484706282



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SharedSlotTest.java
##########
@@ -234,6 +236,45 @@ public void testReleaseIfPhysicalSlotIsAllocated() {
 		assertThat(released.isDone(), is(true));
 	}
 
+	@Test
+	public void tesDuplicatedReturnLogicalSlotFails() {
+		CompletableFuture<PhysicalSlot> slotContextFuture = CompletableFuture
+			.completedFuture(new TestingPhysicalSlot(RP, new AllocationID()));
+		AtomicInteger released = new AtomicInteger(0);
+		SharedSlot sharedSlot = SharedSlotBuilder
+			.newBuilder()
+			.withSlotContextFuture(slotContextFuture)
+			.withReleaseCallback(g -> released.incrementAndGet())

Review comment:
       I think this case is already covered by `testReleaseEmptyDoesNotCallAllocatorReleaseBack` where `returnLogicalSlot` calls the callback and then next `sharedSlot.release` checks that the callback is not called any more. It is not exactly the same but the infinite recursion cannot happen and the recursive setup will be a bit complicated.




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



[GitHub] [flink] zhuzhurk commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r475356048



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/PhysicalSlotRequestBulkCheckerImplTest.java
##########
@@ -0,0 +1,257 @@
+/*
+ * 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.jobmaster.slotpool;
+
+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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.clock.ManualClock;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+import static org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotTestUtils.createPhysicalSlot;
+import static org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotTestUtils.occupyPhysicalSlot;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for {@link PhysicalSlotRequestBulkCheckerImpl}.
+ */
+public class PhysicalSlotRequestBulkCheckerImplTest extends TestLogger {
+
+	private static final Time TIMEOUT = Time.milliseconds(100L);
+
+	private static ScheduledExecutorService singleThreadScheduledExecutorService;
+
+	private static ComponentMainThreadExecutor mainThreadExecutor;
+
+	private final ManualClock clock = new ManualClock();
+
+	private PhysicalSlotRequestBulkCheckerImpl bulkChecker;
+
+	private Set<PhysicalSlot> slots;
+
+	private Supplier<Set<SlotInfo>> slotsRetriever;
+
+	@BeforeClass
+	public static void setupClass() {
+		singleThreadScheduledExecutorService = Executors.newSingleThreadScheduledExecutor();
+		mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forSingleThreadExecutor(singleThreadScheduledExecutorService);
+	}
+
+	@AfterClass
+	public static void teardownClass() {
+		if (singleThreadScheduledExecutorService != null) {
+			singleThreadScheduledExecutorService.shutdownNow();
+		}
+	}
+
+	@Before
+	public void setup() throws Exception {
+		slots = new HashSet<>();
+		slotsRetriever = () -> new HashSet<>(slots);
+		bulkChecker = new PhysicalSlotRequestBulkCheckerImpl(slotsRetriever, clock);
+		bulkChecker.start(mainThreadExecutor);
+	}
+
+	@Test
+	public void testPendingBulkIsNotCancelled() throws InterruptedException, ExecutionException {
+		final CompletableFuture<SlotRequestId> cancellationFuture = new CompletableFuture<>();
+		final PhysicalSlotRequestBulk bulk = createPhysicalSlotRequestBulkWithCancellationFuture(cancellationFuture, new SlotRequestId());
+		bulkChecker.schedulePendingRequestBulkTimeoutCheck(bulk, TIMEOUT);
+		checkNotCancelledAfter(cancellationFuture, 2 * TIMEOUT.toMilliseconds());
+	}
+
+	@Test
+	public void testFulfilledBulkIsNotCancelled() throws InterruptedException, ExecutionException {
+		final CompletableFuture<SlotRequestId> cancellationFuture = new CompletableFuture<>();
+		final PhysicalSlotRequestBulk bulk = createPhysicalSlotRequestBulkWithCancellationFuture(cancellationFuture, new SlotRequestId());
+		bulkChecker.schedulePendingRequestBulkTimeoutCheck(bulk, TIMEOUT);
+		checkNotCancelledAfter(cancellationFuture, 2  * TIMEOUT.toMilliseconds());
+	}
+
+	private static void checkNotCancelledAfter(
+			CompletableFuture<?> cancellationFuture, long milli) throws ExecutionException, InterruptedException {
+		mainThreadExecutor.schedule(() -> {}, milli, TimeUnit.MILLISECONDS).get();
+		try {
+			cancellationFuture.get(milli, TimeUnit.MILLISECONDS);

Review comment:
       Maybe `checkThat(cancellationFuture.isDone(), is(false))`? Otherwise it will actually wait for `2 * milli` milliseconds in this method.




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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r484706282



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SharedSlotTest.java
##########
@@ -234,6 +236,45 @@ public void testReleaseIfPhysicalSlotIsAllocated() {
 		assertThat(released.isDone(), is(true));
 	}
 
+	@Test
+	public void tesDuplicatedReturnLogicalSlotFails() {
+		CompletableFuture<PhysicalSlot> slotContextFuture = CompletableFuture
+			.completedFuture(new TestingPhysicalSlot(RP, new AllocationID()));
+		AtomicInteger released = new AtomicInteger(0);
+		SharedSlot sharedSlot = SharedSlotBuilder
+			.newBuilder()
+			.withSlotContextFuture(slotContextFuture)
+			.withReleaseCallback(g -> released.incrementAndGet())

Review comment:
       I think this case is already covered by `testReleaseEmptyDoesNotCallAllocatorReleaseBack` where `returnLogicalSlot` calls the callback and then next `sharedSlot.release` checks that the callback is not called any more. It is not exactly the same but the infinite recursion cannot happen and the setup will be a bit complicated.




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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * a96b2db52a0db507e0077266c8e9cb947413e1ba Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176) 
   * d34648bab5e9a642c97db6e98f80f8059500c145 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r481844415



##########
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));
+	}
+
+	void markFulfilled(ExecutionSlotSharingGroup group, AllocationID allocationID) {
+		pendingRequests.remove(group);
+		fulfilledRequests.put(group, allocationID);
+	}
+
+	void clear() {
+		pendingRequests.clear();

Review comment:
       It is written where it is used atm.
   The class specifically exists for the `SlotSharingExecutionSlotAllocator`.
   I can rename the method to `clearPendingRequests` and add a doc comment




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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202",
       "triggerID" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219",
       "triggerID" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6299",
       "triggerID" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6307",
       "triggerID" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "23cbf376a6cc07924cf9101a95962a34359c1b76",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "23cbf376a6cc07924cf9101a95962a34359c1b76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8dde2eb2e2bb41c213d23f874be8392365f2c4f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6328",
       "triggerID" : "8dde2eb2e2bb41c213d23f874be8392365f2c4f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fe8cc1fc15e29ede254333ae186eb0b6cb62d5a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9fe8cc1fc15e29ede254333ae186eb0b6cb62d5a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dbed0f154c63a16b28530d8d1a5ff351ea64f3d0",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6331",
       "triggerID" : "dbed0f154c63a16b28530d8d1a5ff351ea64f3d0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * 23cbf376a6cc07924cf9101a95962a34359c1b76 UNKNOWN
   * 9fe8cc1fc15e29ede254333ae186eb0b6cb62d5a UNKNOWN
   * dbed0f154c63a16b28530d8d1a5ff351ea64f3d0 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6331) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202",
       "triggerID" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219",
       "triggerID" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6299",
       "triggerID" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * 7b527d1937d410eaa7de59479915074ca02e6fbd Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6299) 
   * d01d63a8d22b82b6b8168d3abed0505191d655e6 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r483577436



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -108,27 +122,30 @@
 
 		SharedSlotProfileRetriever sharedSlotProfileRetriever = sharedSlotProfileRetrieverFactory
 			.createFromBulk(new HashSet<>(executionVertexIds));

Review comment:
       Indeed, this can result in suboptimal behaviour. I created an issue to investigate this: [FLINK-19142](https://issues.apache.org/jira/browse/FLINK-19142).




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



[GitHub] [flink] flinkbot commented on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675087316


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit b54bb16026ebc704e7dbab97bba1f18e0438264b (Mon Aug 17 20:09:11 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7e07e70b62850e70644e35b18c9168426371862c Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794) 
   * 9e7d2cb792556814b91b8f580d9a1cbe98433274 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202",
       "triggerID" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219",
       "triggerID" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6299",
       "triggerID" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6307",
       "triggerID" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "23cbf376a6cc07924cf9101a95962a34359c1b76",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "23cbf376a6cc07924cf9101a95962a34359c1b76",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8dde2eb2e2bb41c213d23f874be8392365f2c4f9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6328",
       "triggerID" : "8dde2eb2e2bb41c213d23f874be8392365f2c4f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9fe8cc1fc15e29ede254333ae186eb0b6cb62d5a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9fe8cc1fc15e29ede254333ae186eb0b6cb62d5a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dbed0f154c63a16b28530d8d1a5ff351ea64f3d0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "dbed0f154c63a16b28530d8d1a5ff351ea64f3d0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * d01d63a8d22b82b6b8168d3abed0505191d655e6 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6307) 
   * 23cbf376a6cc07924cf9101a95962a34359c1b76 UNKNOWN
   * 8dde2eb2e2bb41c213d23f874be8392365f2c4f9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6328) 
   * 9fe8cc1fc15e29ede254333ae186eb0b6cb62d5a UNKNOWN
   * dbed0f154c63a16b28530d8d1a5ff351ea64f3d0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b54bb16026ebc704e7dbab97bba1f18e0438264b Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r484436699



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -161,158 +173,66 @@ private SharedSlot getOrAllocateSharedSlot(
 					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
 						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
 					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
-				return new SharedSlot(physicalSlotRequestId, physicalSlotResourceProfile, group, physicalSlotFuture);
+				return new SharedSlot(
+					physicalSlotRequestId,
+					physicalSlotResourceProfile,
+					group,
+					physicalSlotFuture,
+					slotWillBeOccupiedIndefinitely,
+					this::releaseSharedSlot);
 			});
 	}
 
+	private void releaseSharedSlot(ExecutionSlotSharingGroup executionSlotSharingGroup) {
+		SharedSlot slot = sharedSlots.remove(executionSlotSharingGroup);
+		if (slot != null) {
+			slotProvider.cancelSlotRequest(
+				slot.getPhysicalSlotRequestId(),
+				new FlinkException("Slot is being returned from SlotSharingExecutionSlotAllocator."));
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup {} to release", executionSlotSharingGroup);

Review comment:
       I think we could try to eliminate the duplicated `SlotSharingExecutionSlotAllocator#releaseSharedSlot` call by adding the `!isEmpty` check at the end of `SharedSlot#release`:
   ```
   if (!isEmpty()) {
   	requestedLogicalSlots.clear();
   	releaseCallback.accept(executionSlotSharingGroup);
   }
   ```
    and by hardening `SharedSlot#removeLogicalSlotRequest` with:
   ```
   Preconditions.checkState(
   	requestedLogicalSlots.removeKeyB(logicalSlotRequestId) != null,
   	"Trying to remove a logical slot request which has been either already removed or never created.");
   ```
   wdyt?




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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202",
       "triggerID" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219",
       "triggerID" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * db1607ead33ae1b60634c1162ba8247a9957f0f4 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] azagrebin commented on a change in pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
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.
   I think this is not so important 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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9e7d2cb792556814b91b8f580d9a1cbe98433274 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #13181: [FLINK-18957] Implement logical request bulk tracking in SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13181:
URL: https://github.com/apache/flink/pull/13181#issuecomment-675091412


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5631",
       "triggerID" : "b54bb16026ebc704e7dbab97bba1f18e0438264b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5671",
       "triggerID" : "347df14dd8621cf7922f723aa5faf96a4f4fc1c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e07e70b62850e70644e35b18c9168426371862c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5794",
       "triggerID" : "7e07e70b62850e70644e35b18c9168426371862c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5820",
       "triggerID" : "9e7d2cb792556814b91b8f580d9a1cbe98433274",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7553689356882de1ffe606400d1255d1d757bc4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5926",
       "triggerID" : "bce753ff7f4da2cffa295bd4007517af4c5697d8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5939",
       "triggerID" : "18f88af3b438b13e9a240efd2b4979f841d2b978",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6176",
       "triggerID" : "a96b2db52a0db507e0077266c8e9cb947413e1ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6202",
       "triggerID" : "d34648bab5e9a642c97db6e98f80f8059500c145",
       "triggerType" : "PUSH"
     }, {
       "hash" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6219",
       "triggerID" : "db1607ead33ae1b60634c1162ba8247a9957f0f4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6299",
       "triggerID" : "7b527d1937d410eaa7de59479915074ca02e6fbd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6307",
       "triggerID" : "d01d63a8d22b82b6b8168d3abed0505191d655e6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7553689356882de1ffe606400d1255d1d757bc4 UNKNOWN
   * 7b527d1937d410eaa7de59479915074ca02e6fbd Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6299) 
   * d01d63a8d22b82b6b8168d3abed0505191d655e6 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6307) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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