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/05 16:41:20 UTC

[GitHub] [flink] azagrebin opened a new pull request #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   `SlotSharingExecutionSlotAllocator` maintains a `SharedSlot` for each `ExecutionSlotSharingGroup`. `SlotSharingExecutionSlotAllocator` allocates physical slots for `SharedSlot(s)` and then allocates logical slots from it for scheduled tasks.
   
   The physical slot is lazily allocated for a `SharedSlot`, upon any hosted subtask asking for the `SharedSlot`. Each subsequent sharing subtask allocates a logical slot from the `SharedSlot`. The SharedSlot/physical slot can be released only if all the requested logical slots are released or canceled.
   
   When `SlotSharingExecutionSlotAllocator` receives a set of tasks to allocate slots for, it does the following:
   * Map the tasks to ExecutionSlotSharingGroup(s)
   * Check which ExecutionSlotSharingGroup(s) already have SharedSlot(s)
   * For all involved ExecutionSlotSharingGroup(s) which do not have a SharedSlot yet:
     * Create a SlotProfile future by MergingSharedSlotProfileRetriever and then
     * Allocate a physical slot from the PhysicalSlotProvider
     * Create SharedSlot based on the returned physical slot futures
     * Allocate logical slot futures for the tasks from all corresponding SharedSlot(s).
   * If physical slot future fails, cancel its pending logical slot requests within the SharedSlot
   * Generates SlotExecutionVertexAssignment(s)  based on the logical slot futures and returns the results.


----------------------------------------------------------------
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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/util/DualKeyLinkedMapTest.java
##########
@@ -16,9 +16,10 @@
  * limitations under the License.
  */
 
-package org.apache.flink.runtime.jobmaster.slotpool;
+package org.apache.flink.runtime.util;
 
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.util.DualKeyLinkedMap;

Review comment:
       This import is not needed anymore.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -0,0 +1,291 @@
+/*
+ * 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.SlotProfile;
+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.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.util.DualKeyLinkedMap;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Allocates {@link LogicalSlot}s from physical shared slots.
+ *
+ * <p>The allocator maintains a shared slot for each {@link ExecutionSlotSharingGroup}.
+ * It allocates a physical slot for the shared slot and then allocates logical slots from it for scheduled tasks.
+ * The physical slot is lazily allocated for a shared slot, upon any hosted subtask asking for the shared slot.
+ * Each subsequent sharing subtask allocates a logical slot from the existing shared slot. The shared/physical slot
+ * can be released only if all the requested logical slots are released or canceled.
+ */
+class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
+	private static final Logger LOG = LoggerFactory.getLogger(SlotSharingExecutionSlotAllocator.class);
+
+	private final PhysicalSlotProvider slotProvider;
+
+	private final boolean slotWillBeOccupiedIndefinitely;
+
+	private final SlotSharingStrategy slotSharingStrategy;
+
+	private final Map<ExecutionSlotSharingGroup, SharedSlot> sharedSlots;
+
+	private final SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory;
+
+	SlotSharingExecutionSlotAllocator(
+			PhysicalSlotProvider slotProvider,
+			boolean slotWillBeOccupiedIndefinitely,
+			SlotSharingStrategy slotSharingStrategy,
+			SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory) {
+		this.slotProvider = slotProvider;
+		this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+		this.slotSharingStrategy = slotSharingStrategy;
+		this.sharedSlotProfileRetrieverFactory = sharedSlotProfileRetrieverFactory;
+		this.sharedSlots = new IdentityHashMap<>();
+	}
+
+	/**
+	 * Creates logical {@link SlotExecutionVertexAssignment}s from physical shared slots.
+	 *
+	 * <p>The allocation has the following steps:
+	 * <ol>
+	 *  <li>Map the executions to {@link ExecutionSlotSharingGroup}s using {@link SlotSharingStrategy}</li>
+	 *  <li>Check which {@link ExecutionSlotSharingGroup}s already have shared slot</li>
+	 *  <li>For all involved {@link ExecutionSlotSharingGroup}s which do not have a shared slot yet:</li>
+	 *  <li>Create a {@link SlotProfile} future using {@link SharedSlotProfileRetriever} and then</li>
+	 *  <li>Allocate a physical slot from the {@link PhysicalSlotProvider}</li>
+	 *  <li>Create a shared slot based on the returned physical slot futures</li>
+	 *  <li>Allocate logical slot futures for the executions from all corresponding shared slots.</li>
+	 *  <li>If a physical slot request fails, associated logical slot requests are canceled within the shared slot</li>
+	 *  <li>Generate {@link SlotExecutionVertexAssignment}s based on the logical slot futures and returns the results.</li>
+	 * </ol>
+	 *
+	 * @param executionVertexSchedulingRequirements the requirements for scheduling the executions.
+	 */
+	@Override
+	public List<SlotExecutionVertexAssignment> allocateSlotsFor(
+			List<ExecutionVertexSchedulingRequirements> executionVertexSchedulingRequirements) {
+		List<ExecutionVertexID> executionVertexIds = executionVertexSchedulingRequirements
+			.stream()
+			.map(ExecutionVertexSchedulingRequirements::getExecutionVertexId)
+			.collect(Collectors.toList());
+
+		SharedSlotProfileRetriever sharedSlotProfileRetriever = sharedSlotProfileRetrieverFactory
+			.createFromBulk(new HashSet<>(executionVertexIds));
+		Map<ExecutionVertexID, SlotExecutionVertexAssignment> assignments = executionVertexIds
+			.stream()
+			.collect(Collectors.groupingBy(slotSharingStrategy::getExecutionSlotSharingGroup))
+			.entrySet()
+			.stream()
+			.flatMap(entry -> allocateLogicalSlotsFromSharedSlot(sharedSlotProfileRetriever, entry.getKey(), entry.getValue()))
+			.collect(Collectors.toMap(SlotExecutionVertexAssignment::getExecutionVertexId, a -> a));
+
+		return executionVertexIds.stream().map(assignments::get).collect(Collectors.toList());
+	}
+
+	@Override
+	public void cancel(ExecutionVertexID executionVertexId) {
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			slotSharingStrategy.getExecutionSlotSharingGroup(executionVertexId);
+		Preconditions.checkNotNull(
+			executionSlotSharingGroup,
+			"There is no ExecutionSlotSharingGroup for ExecutionVertexID " + executionVertexId);
+		SharedSlot slot = sharedSlots.get(executionSlotSharingGroup);
+		if (slot != null) {
+			slot.cancelLogicalSlotRequest(executionVertexId);
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup of ExecutionVertexID {}", executionVertexId);
+		}
+	}
+
+	private Stream<SlotExecutionVertexAssignment> allocateLogicalSlotsFromSharedSlot(
+			SharedSlotProfileRetriever sharedSlotProfileRetriever,
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			Collection<ExecutionVertexID> executions) {
+		SharedSlot sharedSlot = getOrAllocateSharedSlot(executionSlotSharingGroup, sharedSlotProfileRetriever);
+		return executions
+			.stream()
+			.map(execution -> new SlotExecutionVertexAssignment(execution, sharedSlot.allocateLogicalSlot(execution)));
+	}
+
+	private SharedSlot getOrAllocateSharedSlot(
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			SharedSlotProfileRetriever sharedSlotProfileRetriever) {
+		return sharedSlots
+			.computeIfAbsent(executionSlotSharingGroup, group -> {
+				SlotRequestId physicalSlotRequestId = new SlotRequestId();
+				CompletableFuture<PhysicalSlot> physicalSlotFuture = sharedSlotProfileRetriever
+					.getSlotProfileFuture(group)
+					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
+						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
+					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
+				return new SharedSlot(physicalSlotRequestId, group, physicalSlotFuture);
+			});
+	}
+
+	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));
+			SlotRequestId logicalSlotRequestId = new SlotRequestId();
+			String logMessageBase = getLogicalSlotString(logicalSlotRequestId, executionVertexId);
+			LOG.debug("Request a {}", logMessageBase);
+
+			CompletableFuture<SingleLogicalSlot> 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
+			// in requestedLogicalSlots and eventually in sharedSlots
+			logicalSlotFuture.exceptionally(cause -> {
+				LOG.debug("Failed {}", logMessageBase);
+				cancelLogicalSlotRequest(logicalSlotRequestId);
+				return null;
+			});
+
+			return logicalSlotFuture.thenApply(Function.identity());
+		}
+
+		private SingleLogicalSlot createLogicalSlot(PhysicalSlot physicalSlot, SlotRequestId logicalSlotRequestId) {
+			return new SingleLogicalSlot(
+				logicalSlotRequestId,
+				physicalSlot,
+				null,
+				Locality.UNKNOWN,
+				this,
+				slotWillBeOccupiedIndefinitely);
+		}
+
+		void cancelLogicalSlotRequest(ExecutionVertexID executionVertexID) {
+			cancelLogicalSlotRequest(requestedLogicalSlots.getKeyBByKeyA(executionVertexID));
+		}
+
+		void cancelLogicalSlotRequest(SlotRequestId logicalSlotRequestId) {
+			CompletableFuture<SingleLogicalSlot> logicalSlotFuture = requestedLogicalSlots.getValueByKeyB(logicalSlotRequestId);
+			if (logicalSlotFuture != null) {
+				LOG.debug("Cancel {}", getLogicalSlotString(logicalSlotRequestId));
+				logicalSlotFuture.cancel(false);
+				requestedLogicalSlots.removeKeyB(logicalSlotRequestId);
+			} else {
+				LOG.debug("No SlotExecutionVertexAssignment for logical {} from physical %s", logicalSlotRequestId);
+			}
+			removeSharedSlotIfAllLogicalDone();
+		}
+
+		private void removeSharedSlotIfAllLogicalDone() {
+			if (requestedLogicalSlots.values().isEmpty()) {
+				sharedSlots.remove(executionSlotSharingGroup);
+				slotProvider.cancelSlotRequest(
+					physicalSlotRequestId,
+					new FlinkException("Slot is being returned from SlotSharingExecutionSlotAllocator."));
+			}
+		}
+
+		@Override
+		public void returnLogicalSlot(LogicalSlot logicalSlot) {
+			cancelLogicalSlotRequest(logicalSlot.getSlotRequestId());
+		}
+
+		@Override
+		public void release(Throwable cause) {
+			Preconditions.checkState(
+				slotContextFuture.isDone(),
+				"Realising of the shared slot is expected only from its successfully allocated physical slot ({})",

Review comment:
       `Realising` -> `Releasing`




----------------------------------------------------------------
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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8a735691f03d9abd842d7917303ccf9c9fe21e56 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212) 
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 708450b388222f8c266604429e604327bce83491 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 pull request #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   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 a change in pull request #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.SlotProfile;
+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.DualKeyLinkedMap;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Allocates {@link LogicalSlot}s from physical shared slots.
+ *
+ * <p>The allocator maintains a shared slot for each {@link ExecutionSlotSharingGroup}.
+ * It allocates a physical slot for the shared slot and then allocates logical slots from it for scheduled tasks.
+ * The physical slot is lazily allocated for a shared slot, upon any hosted subtask asking for the shared slot.
+ * Each subsequent sharing subtask allocates a logical slot from the existing shared slot. The shared/physical slot
+ * can be released only if all the requested logical slots are released or canceled.
+ */
+class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
+	private static final Logger LOG = LoggerFactory.getLogger(SlotSharingExecutionSlotAllocator.class);
+
+	private final PhysicalSlotProvider slotProvider;
+
+	private final boolean slotWillBeOccupiedIndefinitely;
+
+	private final SlotSharingStrategy slotSharingStrategy;
+
+	private final Map<ExecutionSlotSharingGroup, SharedSlot> sharedSlots;
+
+	private final SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory;
+
+	SlotSharingExecutionSlotAllocator(
+			PhysicalSlotProvider slotProvider,
+			boolean slotWillBeOccupiedIndefinitely,
+			SlotSharingStrategy slotSharingStrategy,
+			SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory) {
+		this.slotProvider = slotProvider;
+		this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+		this.slotSharingStrategy = slotSharingStrategy;
+		this.sharedSlotProfileRetrieverFactory = sharedSlotProfileRetrieverFactory;
+		this.sharedSlots = new HashMap<>();
+	}
+
+	/**
+	 * Creates logical {@link SlotExecutionVertexAssignment}s from physical shared slots.
+	 *
+	 * <p>The allocation has the following steps:
+	 * <ol>
+	 *  <li>Map the executions to {@link ExecutionSlotSharingGroup}s using {@link SlotSharingStrategy}</li>
+	 *  <li>Check which {@link ExecutionSlotSharingGroup}s already have shared slot</li>
+	 *  <li>For all involved {@link ExecutionSlotSharingGroup}s which do not have a shared slot yet:</li>
+	 *  <li>Create a {@link SlotProfile} future using {@link SharedSlotProfileRetriever} and then</li>
+	 *  <li>Allocate a physical slot from the {@link PhysicalSlotProvider}</li>
+	 *  <li>Create a shared slot based on the returned physical slot futures</li>
+	 *  <li>Allocate logical slot futures for the executions from all corresponding shared slots.</li>
+	 *  <li>If a physical slot request fails, associated logical slot requests are canceled within the shared slot</li>
+	 *  <li>Generate {@link SlotExecutionVertexAssignment}s based on the logical slot futures and returns the results.</li>
+	 * </ol>
+	 *
+	 * @param executionVertexSchedulingRequirements the requirements for scheduling the executions.
+	 */
+	@Override
+	public List<SlotExecutionVertexAssignment> allocateSlotsFor(
+			List<ExecutionVertexSchedulingRequirements> executionVertexSchedulingRequirements) {
+		List<ExecutionVertexID> executionVertexIds = executionVertexSchedulingRequirements
+			.stream()
+			.map(ExecutionVertexSchedulingRequirements::getExecutionVertexId)
+			.collect(Collectors.toList());
+
+		SharedSlotProfileRetriever sharedSlotProfileRetriever = sharedSlotProfileRetrieverFactory
+			.createFromBulk(new HashSet<>(executionVertexIds));
+		Map<ExecutionVertexID, SlotExecutionVertexAssignment> assignments = executionVertexIds
+			.stream()
+			.collect(Collectors.groupingBy(slotSharingStrategy::getExecutionSlotSharingGroup))
+			.entrySet()
+			.stream()
+			.flatMap(entry -> allocateLogicalSlotsFromSharedSlot(sharedSlotProfileRetriever, entry.getKey(), entry.getValue()))
+			.collect(Collectors.toMap(SlotExecutionVertexAssignment::getExecutionVertexId, a -> a));
+
+		return executionVertexIds.stream().map(assignments::get).collect(Collectors.toList());
+	}
+
+	@Override
+	public void cancel(ExecutionVertexID executionVertexId) {
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			slotSharingStrategy.getExecutionSlotSharingGroup(executionVertexId);
+		Preconditions.checkNotNull(
+			executionSlotSharingGroup,
+			"There is no ExecutionSlotSharingGroup for ExecutionVertexID " + executionVertexId);
+		SharedSlot slot = sharedSlots.get(executionSlotSharingGroup);
+		if (slot != null) {
+			slot.cancelLogicalSlotRequest(executionVertexId);
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup of ExecutionVertexID {}", executionVertexId);
+		}
+	}
+
+	private Stream<? extends SlotExecutionVertexAssignment> allocateLogicalSlotsFromSharedSlot(
+			SharedSlotProfileRetriever sharedSlotProfileRetriever,
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			Collection<ExecutionVertexID> executions) {
+		SharedSlot sharedSlot = getOrAllocateSharedSlot(executionSlotSharingGroup, sharedSlotProfileRetriever);
+		return executions.stream().map(sharedSlot::createAssignment);
+	}
+
+	private SharedSlot getOrAllocateSharedSlot(
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			SharedSlotProfileRetriever sharedSlotProfileRetriever) {
+		return sharedSlots
+			.computeIfAbsent(executionSlotSharingGroup, group -> {
+				SlotRequestId physicalSlotRequestId = new SlotRequestId();
+				CompletableFuture<PhysicalSlot> physicalSlotFuture = sharedSlotProfileRetriever
+					.getSlotProfileFuture(group)
+					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
+						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
+					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
+				return new SharedSlot(physicalSlotRequestId, group, physicalSlotFuture);
+			});
+	}
+
+	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, SlotExecutionVertexAssignment> 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());
+		}
+
+		SlotExecutionVertexAssignment createAssignment(ExecutionVertexID executionVertexId) {
+			Preconditions.checkArgument(executionSlotSharingGroup.getExecutionVertexIds().contains(executionVertexId));
+			SlotRequestId logicalSlotRequestId = new SlotRequestId();
+			CompletableFuture<LogicalSlot> logicalSlotFuture = allocateLogicalSlot(logicalSlotRequestId, executionVertexId);
+			SlotExecutionVertexAssignment assignment = new SlotExecutionVertexAssignment(executionVertexId, logicalSlotFuture);
+			requestedLogicalSlots.put(
+				executionVertexId,
+				logicalSlotRequestId,
+				assignment);
+			return assignment;
+		}
+
+		private CompletableFuture<LogicalSlot> allocateLogicalSlot(
+				SlotRequestId logicalSlotRequestId,
+				ExecutionVertexID executionVertexId) {
+			String logMessageBase = getLogicalSlotString(logicalSlotRequestId, executionVertexId);
+			LOG.debug("Request a {}", logMessageBase);
+			CompletableFuture<LogicalSlot> logicalSlotFuture = slotContextFuture.thenApply(physicalSlot -> {
+				LOG.debug("Allocated {}", logMessageBase);
+				return new SingleLogicalSlot(
+					logicalSlotRequestId,
+					physicalSlot,
+					null,
+					Locality.UNKNOWN,
+					this,
+					slotWillBeOccupiedIndefinitely);
+			});
+			logicalSlotFuture.exceptionally(cause -> {
+				LOG.debug("Failed {}", logMessageBase);
+				cancelLogicalSlotRequest(logicalSlotRequestId);
+				return null;
+			});
+			return logicalSlotFuture;
+		}
+
+		void cancelLogicalSlotRequest(ExecutionVertexID executionVertexID) {
+			cancelLogicalSlotRequest(requestedLogicalSlots.getKeyBByKeyA(executionVertexID));
+		}
+
+		void cancelLogicalSlotRequest(SlotRequestId logicalSlotRequestId) {
+			SlotExecutionVertexAssignment assignment = requestedLogicalSlots.getValueByKeyB(logicalSlotRequestId);
+			if (assignment != null) {
+				LOG.debug("Cancel {}", getLogicalSlotString(logicalSlotRequestId, assignment.getExecutionVertexId()));
+				assignment.getLogicalSlotFuture().cancel(false);
+				requestedLogicalSlots.removeKeyB(logicalSlotRequestId);
+			} else {
+				LOG.debug("No SlotExecutionVertexAssignment for logical {} from physical %s", logicalSlotRequestId);
+			}
+			if (requestedLogicalSlots.values().isEmpty()) {
+				sharedSlots.remove(executionSlotSharingGroup);
+				slotProvider.cancelSlotRequest(
+					physicalSlotRequestId,
+					new FlinkException("Slot is being returned from SlotSharingExecutionSlotAllocator."));
+			}
+		}
+
+		@Override
+		public void returnLogicalSlot(LogicalSlot logicalSlot) {
+			cancelLogicalSlotRequest(logicalSlot.getSlotRequestId());
+		}
+
+		@Override
+		public void release(Throwable cause) {
+			for (SlotExecutionVertexAssignment assignment : requestedLogicalSlots.values()) {
+				ExecutionVertexID executionVertexId = assignment.getExecutionVertexId();
+				LOG.debug("Release {}", getLogicalSlotString(requestedLogicalSlots.getKeyBByKeyA(executionVertexId), executionVertexId));
+				assignment

Review comment:
       ok, I also expected that the release call only comes from an allocated physical slot, so no need to touch it and logical slots. I was confused because the code in this method does not depend on this assumption.
   I will add the sanity checks to make it explicit.




----------------------------------------------------------------
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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07fd40233580c1641fda58742e015144d767c2fe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07fd40233580c1641fda58742e015144d767c2fe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5256",
       "triggerID" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5276",
       "triggerID" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5290",
       "triggerID" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5304",
       "triggerID" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e98d6b28318bd322f7114bab3b1fdd753936362",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5439",
       "triggerID" : "5e98d6b28318bd322f7114bab3b1fdd753936362",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 07fd40233580c1641fda58742e015144d767c2fe UNKNOWN
   * 0f6489c64935c086c4cc980bc7f5cf858f8544c4 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5304) 
   * 5e98d6b28318bd322f7114bab3b1fdd753936362 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5439) 
   
   <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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07fd40233580c1641fda58742e015144d767c2fe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07fd40233580c1641fda58742e015144d767c2fe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5256",
       "triggerID" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5276",
       "triggerID" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5290",
       "triggerID" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5304",
       "triggerID" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e98d6b28318bd322f7114bab3b1fdd753936362",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5439",
       "triggerID" : "5e98d6b28318bd322f7114bab3b1fdd753936362",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f29ba5bcfd9ee2632ae34f66e635074858c9b37b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f29ba5bcfd9ee2632ae34f66e635074858c9b37b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 07fd40233580c1641fda58742e015144d767c2fe UNKNOWN
   * 5e98d6b28318bd322f7114bab3b1fdd753936362 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5439) 
   * f29ba5bcfd9ee2632ae34f66e635074858c9b37b 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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.SlotProfile;
+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.DualKeyLinkedMap;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Allocates {@link LogicalSlot}s from physical shared slots.
+ *
+ * <p>The allocator maintains a shared slot for each {@link ExecutionSlotSharingGroup}.
+ * It allocates a physical slot for the shared slot and then allocates logical slots from it for scheduled tasks.
+ * The physical slot is lazily allocated for a shared slot, upon any hosted subtask asking for the shared slot.
+ * Each subsequent sharing subtask allocates a logical slot from the existing shared slot. The shared/physical slot
+ * can be released only if all the requested logical slots are released or canceled.
+ */
+class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
+	private static final Logger LOG = LoggerFactory.getLogger(SlotSharingExecutionSlotAllocator.class);
+
+	private final PhysicalSlotProvider slotProvider;
+
+	private final boolean slotWillBeOccupiedIndefinitely;
+
+	private final SlotSharingStrategy slotSharingStrategy;
+
+	private final Map<ExecutionSlotSharingGroup, SharedSlot> sharedSlots;
+
+	private final SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory;
+
+	SlotSharingExecutionSlotAllocator(
+			PhysicalSlotProvider slotProvider,
+			boolean slotWillBeOccupiedIndefinitely,
+			SlotSharingStrategy slotSharingStrategy,
+			SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory) {
+		this.slotProvider = slotProvider;
+		this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+		this.slotSharingStrategy = slotSharingStrategy;
+		this.sharedSlotProfileRetrieverFactory = sharedSlotProfileRetrieverFactory;
+		this.sharedSlots = new HashMap<>();
+	}
+
+	/**
+	 * Creates logical {@link SlotExecutionVertexAssignment}s from physical shared slots.
+	 *
+	 * <p>The allocation has the following steps:
+	 * <ol>
+	 *  <li>Map the executions to {@link ExecutionSlotSharingGroup}s using {@link SlotSharingStrategy}</li>
+	 *  <li>Check which {@link ExecutionSlotSharingGroup}s already have shared slot</li>
+	 *  <li>For all involved {@link ExecutionSlotSharingGroup}s which do not have a shared slot yet:</li>
+	 *  <li>Create a {@link SlotProfile} future using {@link SharedSlotProfileRetriever} and then</li>
+	 *  <li>Allocate a physical slot from the {@link PhysicalSlotProvider}</li>
+	 *  <li>Create a shared slot based on the returned physical slot futures</li>
+	 *  <li>Allocate logical slot futures for the executions from all corresponding shared slots.</li>
+	 *  <li>If a physical slot request fails, associated logical slot requests are canceled within the shared slot</li>
+	 *  <li>Generate {@link SlotExecutionVertexAssignment}s based on the logical slot futures and returns the results.</li>
+	 * </ol>
+	 *
+	 * @param executionVertexSchedulingRequirements the requirements for scheduling the executions.
+	 */
+	@Override
+	public List<SlotExecutionVertexAssignment> allocateSlotsFor(
+			List<ExecutionVertexSchedulingRequirements> executionVertexSchedulingRequirements) {
+		List<ExecutionVertexID> executionVertexIds = executionVertexSchedulingRequirements
+			.stream()
+			.map(ExecutionVertexSchedulingRequirements::getExecutionVertexId)
+			.collect(Collectors.toList());
+
+		SharedSlotProfileRetriever sharedSlotProfileRetriever = sharedSlotProfileRetrieverFactory
+			.createFromBulk(new HashSet<>(executionVertexIds));
+		Map<ExecutionVertexID, SlotExecutionVertexAssignment> assignments = executionVertexIds
+			.stream()
+			.collect(Collectors.groupingBy(slotSharingStrategy::getExecutionSlotSharingGroup))
+			.entrySet()
+			.stream()
+			.flatMap(entry -> allocateLogicalSlotsFromSharedSlot(sharedSlotProfileRetriever, entry.getKey(), entry.getValue()))
+			.collect(Collectors.toMap(SlotExecutionVertexAssignment::getExecutionVertexId, a -> a));
+
+		return executionVertexIds.stream().map(assignments::get).collect(Collectors.toList());
+	}
+
+	@Override
+	public void cancel(ExecutionVertexID executionVertexId) {
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			slotSharingStrategy.getExecutionSlotSharingGroup(executionVertexId);
+		Preconditions.checkNotNull(
+			executionSlotSharingGroup,
+			"There is no ExecutionSlotSharingGroup for ExecutionVertexID " + executionVertexId);
+		SharedSlot slot = sharedSlots.get(executionSlotSharingGroup);
+		if (slot != null) {
+			slot.cancelLogicalSlotRequest(executionVertexId);
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup of ExecutionVertexID {}", executionVertexId);
+		}
+	}
+
+	private Stream<? extends SlotExecutionVertexAssignment> allocateLogicalSlotsFromSharedSlot(
+			SharedSlotProfileRetriever sharedSlotProfileRetriever,
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			Collection<ExecutionVertexID> executions) {
+		SharedSlot sharedSlot = getOrAllocateSharedSlot(executionSlotSharingGroup, sharedSlotProfileRetriever);
+		return executions.stream().map(sharedSlot::createAssignment);
+	}
+
+	private SharedSlot getOrAllocateSharedSlot(
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			SharedSlotProfileRetriever sharedSlotProfileRetriever) {
+		return sharedSlots
+			.computeIfAbsent(executionSlotSharingGroup, group -> {
+				SlotRequestId physicalSlotRequestId = new SlotRequestId();
+				CompletableFuture<PhysicalSlot> physicalSlotFuture = sharedSlotProfileRetriever
+					.getSlotProfileFuture(group)
+					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
+						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
+					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
+				return new SharedSlot(physicalSlotRequestId, group, physicalSlotFuture);
+			});
+	}
+
+	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, SlotExecutionVertexAssignment> requestedLogicalSlots;
+
+		private SharedSlot(
+				SlotRequestId physicalSlotRequestId,
+				ExecutionSlotSharingGroup executionSlotSharingGroup,
+				CompletableFuture<PhysicalSlot> slotContextFuture) {
+			this.physicalSlotRequestId = physicalSlotRequestId;
+			this.executionSlotSharingGroup = executionSlotSharingGroup;
+			this.slotContextFuture = slotContextFuture.thenApply(physicalSlot -> {

Review comment:
       Sorry I made a mistake here. Failed physical slot allocation will result in failed logical slot future, and finally removes the shared slot itself.




----------------------------------------------------------------
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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07fd40233580c1641fda58742e015144d767c2fe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07fd40233580c1641fda58742e015144d767c2fe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5256",
       "triggerID" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 07fd40233580c1641fda58742e015144d767c2fe UNKNOWN
   * 2d0c93809d722ca1325ebec8aa4de38c83cc3f5f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5256) 
   * 7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b 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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07fd40233580c1641fda58742e015144d767c2fe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07fd40233580c1641fda58742e015144d767c2fe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5256",
       "triggerID" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5276",
       "triggerID" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5290",
       "triggerID" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5304",
       "triggerID" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e98d6b28318bd322f7114bab3b1fdd753936362",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5439",
       "triggerID" : "5e98d6b28318bd322f7114bab3b1fdd753936362",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f29ba5bcfd9ee2632ae34f66e635074858c9b37b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5487",
       "triggerID" : "f29ba5bcfd9ee2632ae34f66e635074858c9b37b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ba4f4388232fbb26ce7fe440915f07c7b79f2b15",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5509",
       "triggerID" : "ba4f4388232fbb26ce7fe440915f07c7b79f2b15",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 07fd40233580c1641fda58742e015144d767c2fe UNKNOWN
   * ba4f4388232fbb26ce7fe440915f07c7b79f2b15 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5509) 
   
   <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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07fd40233580c1641fda58742e015144d767c2fe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07fd40233580c1641fda58742e015144d767c2fe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 223e7673843cdc4fc6467ce0ed9ad0ff0630aef5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251) 
   * 07fd40233580c1641fda58742e015144d767c2fe UNKNOWN
   * 2d0c93809d722ca1325ebec8aa4de38c83cc3f5f 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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07fd40233580c1641fda58742e015144d767c2fe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07fd40233580c1641fda58742e015144d767c2fe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5256",
       "triggerID" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5276",
       "triggerID" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5290",
       "triggerID" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5304",
       "triggerID" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e98d6b28318bd322f7114bab3b1fdd753936362",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5e98d6b28318bd322f7114bab3b1fdd753936362",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 07fd40233580c1641fda58742e015144d767c2fe UNKNOWN
   * 0f6489c64935c086c4cc980bc7f5cf858f8544c4 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5304) 
   * 5e98d6b28318bd322f7114bab3b1fdd753936362 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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/DualKeyLinkedMapTest.java
##########
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.jobmaster.slotpool;

Review comment:
       I think this UT should be moved as well if we would move `DualKeyLinkedMap`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/util/DualKeyLinkedMap.java
##########
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.runtime.jobmaster.slotpool;
+package org.apache.flink.util;

Review comment:
       Not pretty sure why we move it to flink-core?




----------------------------------------------------------------
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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.SlotProfile;
+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.DualKeyLinkedMap;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Allocates {@link LogicalSlot}s from physical shared slots.
+ *
+ * <p>The allocator maintains a shared slot for each {@link ExecutionSlotSharingGroup}.
+ * It allocates a physical slot for the shared slot and then allocates logical slots from it for scheduled tasks.
+ * The physical slot is lazily allocated for a shared slot, upon any hosted subtask asking for the shared slot.
+ * Each subsequent sharing subtask allocates a logical slot from the existing shared slot. The shared/physical slot
+ * can be released only if all the requested logical slots are released or canceled.
+ */
+class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
+	private static final Logger LOG = LoggerFactory.getLogger(SlotSharingExecutionSlotAllocator.class);
+
+	private final PhysicalSlotProvider slotProvider;
+
+	private final boolean slotWillBeOccupiedIndefinitely;
+
+	private final SlotSharingStrategy slotSharingStrategy;
+
+	private final Map<ExecutionSlotSharingGroup, SharedSlot> sharedSlots;
+
+	private final SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory;
+
+	SlotSharingExecutionSlotAllocator(

Review comment:
       True, I will also implement the factory




----------------------------------------------------------------
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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07fd40233580c1641fda58742e015144d767c2fe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07fd40233580c1641fda58742e015144d767c2fe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5256",
       "triggerID" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5276",
       "triggerID" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5290",
       "triggerID" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5304",
       "triggerID" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e98d6b28318bd322f7114bab3b1fdd753936362",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5439",
       "triggerID" : "5e98d6b28318bd322f7114bab3b1fdd753936362",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f29ba5bcfd9ee2632ae34f66e635074858c9b37b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5487",
       "triggerID" : "f29ba5bcfd9ee2632ae34f66e635074858c9b37b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ba4f4388232fbb26ce7fe440915f07c7b79f2b15",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ba4f4388232fbb26ce7fe440915f07c7b79f2b15",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 07fd40233580c1641fda58742e015144d767c2fe UNKNOWN
   * f29ba5bcfd9ee2632ae34f66e635074858c9b37b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5487) 
   * ba4f4388232fbb26ce7fe440915f07c7b79f2b15 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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 708450b388222f8c266604429e604327bce83491 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242) 
   
   <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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07fd40233580c1641fda58742e015144d767c2fe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07fd40233580c1641fda58742e015144d767c2fe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5256",
       "triggerID" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5276",
       "triggerID" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5290",
       "triggerID" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5304",
       "triggerID" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 07fd40233580c1641fda58742e015144d767c2fe UNKNOWN
   * 97ee7a057155f9317a7890d405cb1f2ad363c271 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5290) 
   * 0f6489c64935c086c4cc980bc7f5cf858f8544c4 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5304) 
   
   <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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07fd40233580c1641fda58742e015144d767c2fe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07fd40233580c1641fda58742e015144d767c2fe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5256",
       "triggerID" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5276",
       "triggerID" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5290",
       "triggerID" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5304",
       "triggerID" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e98d6b28318bd322f7114bab3b1fdd753936362",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5439",
       "triggerID" : "5e98d6b28318bd322f7114bab3b1fdd753936362",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f29ba5bcfd9ee2632ae34f66e635074858c9b37b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5487",
       "triggerID" : "f29ba5bcfd9ee2632ae34f66e635074858c9b37b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 07fd40233580c1641fda58742e015144d767c2fe UNKNOWN
   * f29ba5bcfd9ee2632ae34f66e635074858c9b37b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5487) 
   
   <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 commented on pull request #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   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 802a68b965a471d2a3251f03c9ba8adf05e94c63 (Wed Aug 05 16:39:55 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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07fd40233580c1641fda58742e015144d767c2fe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07fd40233580c1641fda58742e015144d767c2fe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5256",
       "triggerID" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5276",
       "triggerID" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 07fd40233580c1641fda58742e015144d767c2fe UNKNOWN
   * 7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5276) 
   
   <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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07fd40233580c1641fda58742e015144d767c2fe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07fd40233580c1641fda58742e015144d767c2fe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5256",
       "triggerID" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5276",
       "triggerID" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5290",
       "triggerID" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5304",
       "triggerID" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 07fd40233580c1641fda58742e015144d767c2fe UNKNOWN
   * 0f6489c64935c086c4cc980bc7f5cf858f8544c4 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5304) 
   
   <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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8a735691f03d9abd842d7917303ccf9c9fe21e56 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212) 
   
   <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 edited a comment on pull request #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   I will add tests later.


----------------------------------------------------------------
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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8a735691f03d9abd842d7917303ccf9c9fe21e56 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212) 
   * 2233727954b8a6272c4a772a03bccc08b5e97720 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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 223e7673843cdc4fc6467ce0ed9ad0ff0630aef5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251) 
   
   <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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.SlotProfile;
+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.DualKeyLinkedMap;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Allocates {@link LogicalSlot}s from physical shared slots.
+ *
+ * <p>The allocator maintains a shared slot for each {@link ExecutionSlotSharingGroup}.
+ * It allocates a physical slot for the shared slot and then allocates logical slots from it for scheduled tasks.
+ * The physical slot is lazily allocated for a shared slot, upon any hosted subtask asking for the shared slot.
+ * Each subsequent sharing subtask allocates a logical slot from the existing shared slot. The shared/physical slot
+ * can be released only if all the requested logical slots are released or canceled.
+ */
+class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
+	private static final Logger LOG = LoggerFactory.getLogger(SlotSharingExecutionSlotAllocator.class);
+
+	private final PhysicalSlotProvider slotProvider;
+
+	private final boolean slotWillBeOccupiedIndefinitely;
+
+	private final SlotSharingStrategy slotSharingStrategy;
+
+	private final Map<ExecutionSlotSharingGroup, SharedSlot> sharedSlots;
+
+	private final SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory;
+
+	SlotSharingExecutionSlotAllocator(

Review comment:
       Is it expected to implement the factory of `SlotSharingExecutionSlotAllocator` in another task?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.SlotProfile;
+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.DualKeyLinkedMap;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Allocates {@link LogicalSlot}s from physical shared slots.
+ *
+ * <p>The allocator maintains a shared slot for each {@link ExecutionSlotSharingGroup}.
+ * It allocates a physical slot for the shared slot and then allocates logical slots from it for scheduled tasks.
+ * The physical slot is lazily allocated for a shared slot, upon any hosted subtask asking for the shared slot.
+ * Each subsequent sharing subtask allocates a logical slot from the existing shared slot. The shared/physical slot
+ * can be released only if all the requested logical slots are released or canceled.
+ */
+class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
+	private static final Logger LOG = LoggerFactory.getLogger(SlotSharingExecutionSlotAllocator.class);
+
+	private final PhysicalSlotProvider slotProvider;
+
+	private final boolean slotWillBeOccupiedIndefinitely;
+
+	private final SlotSharingStrategy slotSharingStrategy;
+
+	private final Map<ExecutionSlotSharingGroup, SharedSlot> sharedSlots;
+
+	private final SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory;
+
+	SlotSharingExecutionSlotAllocator(
+			PhysicalSlotProvider slotProvider,
+			boolean slotWillBeOccupiedIndefinitely,
+			SlotSharingStrategy slotSharingStrategy,
+			SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory) {
+		this.slotProvider = slotProvider;
+		this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+		this.slotSharingStrategy = slotSharingStrategy;
+		this.sharedSlotProfileRetrieverFactory = sharedSlotProfileRetrieverFactory;
+		this.sharedSlots = new HashMap<>();

Review comment:
       Maybe `IdentityHashMap` to make it explicit that we judge with the reference id?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.SlotProfile;
+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.DualKeyLinkedMap;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Allocates {@link LogicalSlot}s from physical shared slots.
+ *
+ * <p>The allocator maintains a shared slot for each {@link ExecutionSlotSharingGroup}.
+ * It allocates a physical slot for the shared slot and then allocates logical slots from it for scheduled tasks.
+ * The physical slot is lazily allocated for a shared slot, upon any hosted subtask asking for the shared slot.
+ * Each subsequent sharing subtask allocates a logical slot from the existing shared slot. The shared/physical slot
+ * can be released only if all the requested logical slots are released or canceled.
+ */
+class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
+	private static final Logger LOG = LoggerFactory.getLogger(SlotSharingExecutionSlotAllocator.class);
+
+	private final PhysicalSlotProvider slotProvider;
+
+	private final boolean slotWillBeOccupiedIndefinitely;
+
+	private final SlotSharingStrategy slotSharingStrategy;
+
+	private final Map<ExecutionSlotSharingGroup, SharedSlot> sharedSlots;
+
+	private final SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory;
+
+	SlotSharingExecutionSlotAllocator(
+			PhysicalSlotProvider slotProvider,
+			boolean slotWillBeOccupiedIndefinitely,
+			SlotSharingStrategy slotSharingStrategy,
+			SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory) {
+		this.slotProvider = slotProvider;
+		this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+		this.slotSharingStrategy = slotSharingStrategy;
+		this.sharedSlotProfileRetrieverFactory = sharedSlotProfileRetrieverFactory;
+		this.sharedSlots = new HashMap<>();
+	}
+
+	/**
+	 * Creates logical {@link SlotExecutionVertexAssignment}s from physical shared slots.
+	 *
+	 * <p>The allocation has the following steps:
+	 * <ol>
+	 *  <li>Map the executions to {@link ExecutionSlotSharingGroup}s using {@link SlotSharingStrategy}</li>
+	 *  <li>Check which {@link ExecutionSlotSharingGroup}s already have shared slot</li>
+	 *  <li>For all involved {@link ExecutionSlotSharingGroup}s which do not have a shared slot yet:</li>
+	 *  <li>Create a {@link SlotProfile} future using {@link SharedSlotProfileRetriever} and then</li>
+	 *  <li>Allocate a physical slot from the {@link PhysicalSlotProvider}</li>
+	 *  <li>Create a shared slot based on the returned physical slot futures</li>
+	 *  <li>Allocate logical slot futures for the executions from all corresponding shared slots.</li>
+	 *  <li>If a physical slot request fails, associated logical slot requests are canceled within the shared slot</li>
+	 *  <li>Generate {@link SlotExecutionVertexAssignment}s based on the logical slot futures and returns the results.</li>
+	 * </ol>
+	 *
+	 * @param executionVertexSchedulingRequirements the requirements for scheduling the executions.
+	 */
+	@Override
+	public List<SlotExecutionVertexAssignment> allocateSlotsFor(
+			List<ExecutionVertexSchedulingRequirements> executionVertexSchedulingRequirements) {
+		List<ExecutionVertexID> executionVertexIds = executionVertexSchedulingRequirements
+			.stream()
+			.map(ExecutionVertexSchedulingRequirements::getExecutionVertexId)
+			.collect(Collectors.toList());
+
+		SharedSlotProfileRetriever sharedSlotProfileRetriever = sharedSlotProfileRetrieverFactory
+			.createFromBulk(new HashSet<>(executionVertexIds));
+		Map<ExecutionVertexID, SlotExecutionVertexAssignment> assignments = executionVertexIds
+			.stream()
+			.collect(Collectors.groupingBy(slotSharingStrategy::getExecutionSlotSharingGroup))
+			.entrySet()
+			.stream()
+			.flatMap(entry -> allocateLogicalSlotsFromSharedSlot(sharedSlotProfileRetriever, entry.getKey(), entry.getValue()))
+			.collect(Collectors.toMap(SlotExecutionVertexAssignment::getExecutionVertexId, a -> a));
+
+		return executionVertexIds.stream().map(assignments::get).collect(Collectors.toList());
+	}
+
+	@Override
+	public void cancel(ExecutionVertexID executionVertexId) {
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			slotSharingStrategy.getExecutionSlotSharingGroup(executionVertexId);
+		Preconditions.checkNotNull(
+			executionSlotSharingGroup,
+			"There is no ExecutionSlotSharingGroup for ExecutionVertexID " + executionVertexId);
+		SharedSlot slot = sharedSlots.get(executionSlotSharingGroup);
+		if (slot != null) {
+			slot.cancelLogicalSlotRequest(executionVertexId);
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup of ExecutionVertexID {}", executionVertexId);
+		}
+	}
+
+	private Stream<? extends SlotExecutionVertexAssignment> allocateLogicalSlotsFromSharedSlot(
+			SharedSlotProfileRetriever sharedSlotProfileRetriever,
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			Collection<ExecutionVertexID> executions) {
+		SharedSlot sharedSlot = getOrAllocateSharedSlot(executionSlotSharingGroup, sharedSlotProfileRetriever);
+		return executions.stream().map(sharedSlot::createAssignment);
+	}
+
+	private SharedSlot getOrAllocateSharedSlot(
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			SharedSlotProfileRetriever sharedSlotProfileRetriever) {
+		return sharedSlots
+			.computeIfAbsent(executionSlotSharingGroup, group -> {
+				SlotRequestId physicalSlotRequestId = new SlotRequestId();
+				CompletableFuture<PhysicalSlot> physicalSlotFuture = sharedSlotProfileRetriever
+					.getSlotProfileFuture(group)
+					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
+						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
+					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
+				return new SharedSlot(physicalSlotRequestId, group, physicalSlotFuture);
+			});
+	}
+
+	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, SlotExecutionVertexAssignment> requestedLogicalSlots;
+
+		private SharedSlot(
+				SlotRequestId physicalSlotRequestId,
+				ExecutionSlotSharingGroup executionSlotSharingGroup,
+				CompletableFuture<PhysicalSlot> slotContextFuture) {
+			this.physicalSlotRequestId = physicalSlotRequestId;
+			this.executionSlotSharingGroup = executionSlotSharingGroup;
+			this.slotContextFuture = slotContextFuture.thenApply(physicalSlot -> {

Review comment:
       I think we need to cancel all the pending logical slot futures and remove this shared slot if the allocation fails.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.SlotProfile;
+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.DualKeyLinkedMap;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Allocates {@link LogicalSlot}s from physical shared slots.
+ *
+ * <p>The allocator maintains a shared slot for each {@link ExecutionSlotSharingGroup}.
+ * It allocates a physical slot for the shared slot and then allocates logical slots from it for scheduled tasks.
+ * The physical slot is lazily allocated for a shared slot, upon any hosted subtask asking for the shared slot.
+ * Each subsequent sharing subtask allocates a logical slot from the existing shared slot. The shared/physical slot
+ * can be released only if all the requested logical slots are released or canceled.
+ */
+class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
+	private static final Logger LOG = LoggerFactory.getLogger(SlotSharingExecutionSlotAllocator.class);
+
+	private final PhysicalSlotProvider slotProvider;
+
+	private final boolean slotWillBeOccupiedIndefinitely;
+
+	private final SlotSharingStrategy slotSharingStrategy;
+
+	private final Map<ExecutionSlotSharingGroup, SharedSlot> sharedSlots;
+
+	private final SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory;
+
+	SlotSharingExecutionSlotAllocator(
+			PhysicalSlotProvider slotProvider,
+			boolean slotWillBeOccupiedIndefinitely,
+			SlotSharingStrategy slotSharingStrategy,
+			SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory) {
+		this.slotProvider = slotProvider;
+		this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+		this.slotSharingStrategy = slotSharingStrategy;
+		this.sharedSlotProfileRetrieverFactory = sharedSlotProfileRetrieverFactory;
+		this.sharedSlots = new HashMap<>();
+	}
+
+	/**
+	 * Creates logical {@link SlotExecutionVertexAssignment}s from physical shared slots.
+	 *
+	 * <p>The allocation has the following steps:
+	 * <ol>
+	 *  <li>Map the executions to {@link ExecutionSlotSharingGroup}s using {@link SlotSharingStrategy}</li>
+	 *  <li>Check which {@link ExecutionSlotSharingGroup}s already have shared slot</li>
+	 *  <li>For all involved {@link ExecutionSlotSharingGroup}s which do not have a shared slot yet:</li>
+	 *  <li>Create a {@link SlotProfile} future using {@link SharedSlotProfileRetriever} and then</li>
+	 *  <li>Allocate a physical slot from the {@link PhysicalSlotProvider}</li>
+	 *  <li>Create a shared slot based on the returned physical slot futures</li>
+	 *  <li>Allocate logical slot futures for the executions from all corresponding shared slots.</li>
+	 *  <li>If a physical slot request fails, associated logical slot requests are canceled within the shared slot</li>
+	 *  <li>Generate {@link SlotExecutionVertexAssignment}s based on the logical slot futures and returns the results.</li>
+	 * </ol>
+	 *
+	 * @param executionVertexSchedulingRequirements the requirements for scheduling the executions.
+	 */
+	@Override
+	public List<SlotExecutionVertexAssignment> allocateSlotsFor(
+			List<ExecutionVertexSchedulingRequirements> executionVertexSchedulingRequirements) {
+		List<ExecutionVertexID> executionVertexIds = executionVertexSchedulingRequirements
+			.stream()
+			.map(ExecutionVertexSchedulingRequirements::getExecutionVertexId)
+			.collect(Collectors.toList());
+
+		SharedSlotProfileRetriever sharedSlotProfileRetriever = sharedSlotProfileRetrieverFactory
+			.createFromBulk(new HashSet<>(executionVertexIds));
+		Map<ExecutionVertexID, SlotExecutionVertexAssignment> assignments = executionVertexIds
+			.stream()
+			.collect(Collectors.groupingBy(slotSharingStrategy::getExecutionSlotSharingGroup))
+			.entrySet()
+			.stream()
+			.flatMap(entry -> allocateLogicalSlotsFromSharedSlot(sharedSlotProfileRetriever, entry.getKey(), entry.getValue()))
+			.collect(Collectors.toMap(SlotExecutionVertexAssignment::getExecutionVertexId, a -> a));
+
+		return executionVertexIds.stream().map(assignments::get).collect(Collectors.toList());
+	}
+
+	@Override
+	public void cancel(ExecutionVertexID executionVertexId) {
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			slotSharingStrategy.getExecutionSlotSharingGroup(executionVertexId);
+		Preconditions.checkNotNull(
+			executionSlotSharingGroup,
+			"There is no ExecutionSlotSharingGroup for ExecutionVertexID " + executionVertexId);
+		SharedSlot slot = sharedSlots.get(executionSlotSharingGroup);
+		if (slot != null) {
+			slot.cancelLogicalSlotRequest(executionVertexId);
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup of ExecutionVertexID {}", executionVertexId);
+		}
+	}
+
+	private Stream<? extends SlotExecutionVertexAssignment> allocateLogicalSlotsFromSharedSlot(
+			SharedSlotProfileRetriever sharedSlotProfileRetriever,
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			Collection<ExecutionVertexID> executions) {
+		SharedSlot sharedSlot = getOrAllocateSharedSlot(executionSlotSharingGroup, sharedSlotProfileRetriever);
+		return executions.stream().map(sharedSlot::createAssignment);
+	}
+
+	private SharedSlot getOrAllocateSharedSlot(
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			SharedSlotProfileRetriever sharedSlotProfileRetriever) {
+		return sharedSlots
+			.computeIfAbsent(executionSlotSharingGroup, group -> {
+				SlotRequestId physicalSlotRequestId = new SlotRequestId();
+				CompletableFuture<PhysicalSlot> physicalSlotFuture = sharedSlotProfileRetriever
+					.getSlotProfileFuture(group)
+					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
+						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
+					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
+				return new SharedSlot(physicalSlotRequestId, group, physicalSlotFuture);
+			});
+	}
+
+	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, SlotExecutionVertexAssignment> requestedLogicalSlots;

Review comment:
       How about use `CompletableFuture<SingleLogicalSlot>` instead of `SlotExecutionVertexAssignment`?
   `DualKeyLinkedMap` already enables to find the `SlotRequestId` via `ExecutionVertexID` and vice versa.
   We can also get rid of the type cast `((PhysicalSlot.Payload) logicalSlot)` which is a bit hacky.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.SlotProfile;
+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.DualKeyLinkedMap;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Allocates {@link LogicalSlot}s from physical shared slots.
+ *
+ * <p>The allocator maintains a shared slot for each {@link ExecutionSlotSharingGroup}.
+ * It allocates a physical slot for the shared slot and then allocates logical slots from it for scheduled tasks.
+ * The physical slot is lazily allocated for a shared slot, upon any hosted subtask asking for the shared slot.
+ * Each subsequent sharing subtask allocates a logical slot from the existing shared slot. The shared/physical slot
+ * can be released only if all the requested logical slots are released or canceled.
+ */
+class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
+	private static final Logger LOG = LoggerFactory.getLogger(SlotSharingExecutionSlotAllocator.class);
+
+	private final PhysicalSlotProvider slotProvider;
+
+	private final boolean slotWillBeOccupiedIndefinitely;
+
+	private final SlotSharingStrategy slotSharingStrategy;
+
+	private final Map<ExecutionSlotSharingGroup, SharedSlot> sharedSlots;
+
+	private final SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory;
+
+	SlotSharingExecutionSlotAllocator(
+			PhysicalSlotProvider slotProvider,
+			boolean slotWillBeOccupiedIndefinitely,
+			SlotSharingStrategy slotSharingStrategy,
+			SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory) {
+		this.slotProvider = slotProvider;
+		this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+		this.slotSharingStrategy = slotSharingStrategy;
+		this.sharedSlotProfileRetrieverFactory = sharedSlotProfileRetrieverFactory;
+		this.sharedSlots = new HashMap<>();
+	}
+
+	/**
+	 * Creates logical {@link SlotExecutionVertexAssignment}s from physical shared slots.
+	 *
+	 * <p>The allocation has the following steps:
+	 * <ol>
+	 *  <li>Map the executions to {@link ExecutionSlotSharingGroup}s using {@link SlotSharingStrategy}</li>
+	 *  <li>Check which {@link ExecutionSlotSharingGroup}s already have shared slot</li>
+	 *  <li>For all involved {@link ExecutionSlotSharingGroup}s which do not have a shared slot yet:</li>
+	 *  <li>Create a {@link SlotProfile} future using {@link SharedSlotProfileRetriever} and then</li>
+	 *  <li>Allocate a physical slot from the {@link PhysicalSlotProvider}</li>
+	 *  <li>Create a shared slot based on the returned physical slot futures</li>
+	 *  <li>Allocate logical slot futures for the executions from all corresponding shared slots.</li>
+	 *  <li>If a physical slot request fails, associated logical slot requests are canceled within the shared slot</li>
+	 *  <li>Generate {@link SlotExecutionVertexAssignment}s based on the logical slot futures and returns the results.</li>
+	 * </ol>
+	 *
+	 * @param executionVertexSchedulingRequirements the requirements for scheduling the executions.
+	 */
+	@Override
+	public List<SlotExecutionVertexAssignment> allocateSlotsFor(
+			List<ExecutionVertexSchedulingRequirements> executionVertexSchedulingRequirements) {
+		List<ExecutionVertexID> executionVertexIds = executionVertexSchedulingRequirements
+			.stream()
+			.map(ExecutionVertexSchedulingRequirements::getExecutionVertexId)
+			.collect(Collectors.toList());
+
+		SharedSlotProfileRetriever sharedSlotProfileRetriever = sharedSlotProfileRetrieverFactory
+			.createFromBulk(new HashSet<>(executionVertexIds));
+		Map<ExecutionVertexID, SlotExecutionVertexAssignment> assignments = executionVertexIds
+			.stream()
+			.collect(Collectors.groupingBy(slotSharingStrategy::getExecutionSlotSharingGroup))
+			.entrySet()
+			.stream()
+			.flatMap(entry -> allocateLogicalSlotsFromSharedSlot(sharedSlotProfileRetriever, entry.getKey(), entry.getValue()))
+			.collect(Collectors.toMap(SlotExecutionVertexAssignment::getExecutionVertexId, a -> a));
+
+		return executionVertexIds.stream().map(assignments::get).collect(Collectors.toList());
+	}
+
+	@Override
+	public void cancel(ExecutionVertexID executionVertexId) {
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			slotSharingStrategy.getExecutionSlotSharingGroup(executionVertexId);
+		Preconditions.checkNotNull(
+			executionSlotSharingGroup,
+			"There is no ExecutionSlotSharingGroup for ExecutionVertexID " + executionVertexId);
+		SharedSlot slot = sharedSlots.get(executionSlotSharingGroup);
+		if (slot != null) {
+			slot.cancelLogicalSlotRequest(executionVertexId);
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup of ExecutionVertexID {}", executionVertexId);
+		}
+	}
+
+	private Stream<? extends SlotExecutionVertexAssignment> allocateLogicalSlotsFromSharedSlot(

Review comment:
       `Stream<? extends SlotExecutionVertexAssignment>` can simply be `Stream<SlotExecutionVertexAssignment>`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.SlotProfile;
+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.DualKeyLinkedMap;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Allocates {@link LogicalSlot}s from physical shared slots.
+ *
+ * <p>The allocator maintains a shared slot for each {@link ExecutionSlotSharingGroup}.
+ * It allocates a physical slot for the shared slot and then allocates logical slots from it for scheduled tasks.
+ * The physical slot is lazily allocated for a shared slot, upon any hosted subtask asking for the shared slot.
+ * Each subsequent sharing subtask allocates a logical slot from the existing shared slot. The shared/physical slot
+ * can be released only if all the requested logical slots are released or canceled.
+ */
+class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
+	private static final Logger LOG = LoggerFactory.getLogger(SlotSharingExecutionSlotAllocator.class);
+
+	private final PhysicalSlotProvider slotProvider;
+
+	private final boolean slotWillBeOccupiedIndefinitely;
+
+	private final SlotSharingStrategy slotSharingStrategy;
+
+	private final Map<ExecutionSlotSharingGroup, SharedSlot> sharedSlots;
+
+	private final SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory;
+
+	SlotSharingExecutionSlotAllocator(
+			PhysicalSlotProvider slotProvider,
+			boolean slotWillBeOccupiedIndefinitely,
+			SlotSharingStrategy slotSharingStrategy,
+			SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory) {
+		this.slotProvider = slotProvider;
+		this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+		this.slotSharingStrategy = slotSharingStrategy;
+		this.sharedSlotProfileRetrieverFactory = sharedSlotProfileRetrieverFactory;
+		this.sharedSlots = new HashMap<>();
+	}
+
+	/**
+	 * Creates logical {@link SlotExecutionVertexAssignment}s from physical shared slots.
+	 *
+	 * <p>The allocation has the following steps:
+	 * <ol>
+	 *  <li>Map the executions to {@link ExecutionSlotSharingGroup}s using {@link SlotSharingStrategy}</li>
+	 *  <li>Check which {@link ExecutionSlotSharingGroup}s already have shared slot</li>
+	 *  <li>For all involved {@link ExecutionSlotSharingGroup}s which do not have a shared slot yet:</li>
+	 *  <li>Create a {@link SlotProfile} future using {@link SharedSlotProfileRetriever} and then</li>
+	 *  <li>Allocate a physical slot from the {@link PhysicalSlotProvider}</li>
+	 *  <li>Create a shared slot based on the returned physical slot futures</li>
+	 *  <li>Allocate logical slot futures for the executions from all corresponding shared slots.</li>
+	 *  <li>If a physical slot request fails, associated logical slot requests are canceled within the shared slot</li>
+	 *  <li>Generate {@link SlotExecutionVertexAssignment}s based on the logical slot futures and returns the results.</li>
+	 * </ol>
+	 *
+	 * @param executionVertexSchedulingRequirements the requirements for scheduling the executions.
+	 */
+	@Override
+	public List<SlotExecutionVertexAssignment> allocateSlotsFor(
+			List<ExecutionVertexSchedulingRequirements> executionVertexSchedulingRequirements) {
+		List<ExecutionVertexID> executionVertexIds = executionVertexSchedulingRequirements
+			.stream()
+			.map(ExecutionVertexSchedulingRequirements::getExecutionVertexId)
+			.collect(Collectors.toList());
+
+		SharedSlotProfileRetriever sharedSlotProfileRetriever = sharedSlotProfileRetrieverFactory
+			.createFromBulk(new HashSet<>(executionVertexIds));
+		Map<ExecutionVertexID, SlotExecutionVertexAssignment> assignments = executionVertexIds
+			.stream()
+			.collect(Collectors.groupingBy(slotSharingStrategy::getExecutionSlotSharingGroup))
+			.entrySet()
+			.stream()
+			.flatMap(entry -> allocateLogicalSlotsFromSharedSlot(sharedSlotProfileRetriever, entry.getKey(), entry.getValue()))
+			.collect(Collectors.toMap(SlotExecutionVertexAssignment::getExecutionVertexId, a -> a));
+
+		return executionVertexIds.stream().map(assignments::get).collect(Collectors.toList());
+	}
+
+	@Override
+	public void cancel(ExecutionVertexID executionVertexId) {
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			slotSharingStrategy.getExecutionSlotSharingGroup(executionVertexId);
+		Preconditions.checkNotNull(
+			executionSlotSharingGroup,
+			"There is no ExecutionSlotSharingGroup for ExecutionVertexID " + executionVertexId);
+		SharedSlot slot = sharedSlots.get(executionSlotSharingGroup);
+		if (slot != null) {
+			slot.cancelLogicalSlotRequest(executionVertexId);
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup of ExecutionVertexID {}", executionVertexId);
+		}
+	}
+
+	private Stream<? extends SlotExecutionVertexAssignment> allocateLogicalSlotsFromSharedSlot(
+			SharedSlotProfileRetriever sharedSlotProfileRetriever,
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			Collection<ExecutionVertexID> executions) {
+		SharedSlot sharedSlot = getOrAllocateSharedSlot(executionSlotSharingGroup, sharedSlotProfileRetriever);
+		return executions.stream().map(sharedSlot::createAssignment);
+	}
+
+	private SharedSlot getOrAllocateSharedSlot(
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			SharedSlotProfileRetriever sharedSlotProfileRetriever) {
+		return sharedSlots
+			.computeIfAbsent(executionSlotSharingGroup, group -> {
+				SlotRequestId physicalSlotRequestId = new SlotRequestId();
+				CompletableFuture<PhysicalSlot> physicalSlotFuture = sharedSlotProfileRetriever
+					.getSlotProfileFuture(group)
+					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
+						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
+					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
+				return new SharedSlot(physicalSlotRequestId, group, physicalSlotFuture);
+			});
+	}
+
+	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, SlotExecutionVertexAssignment> 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());
+		}
+
+		SlotExecutionVertexAssignment createAssignment(ExecutionVertexID executionVertexId) {
+			Preconditions.checkArgument(executionSlotSharingGroup.getExecutionVertexIds().contains(executionVertexId));
+			SlotRequestId logicalSlotRequestId = new SlotRequestId();
+			CompletableFuture<LogicalSlot> logicalSlotFuture = allocateLogicalSlot(logicalSlotRequestId, executionVertexId);
+			SlotExecutionVertexAssignment assignment = new SlotExecutionVertexAssignment(executionVertexId, logicalSlotFuture);
+			requestedLogicalSlots.put(
+				executionVertexId,
+				logicalSlotRequestId,
+				assignment);
+			return assignment;
+		}
+
+		private CompletableFuture<LogicalSlot> allocateLogicalSlot(
+				SlotRequestId logicalSlotRequestId,
+				ExecutionVertexID executionVertexId) {
+			String logMessageBase = getLogicalSlotString(logicalSlotRequestId, executionVertexId);
+			LOG.debug("Request a {}", logMessageBase);
+			CompletableFuture<LogicalSlot> logicalSlotFuture = slotContextFuture.thenApply(physicalSlot -> {
+				LOG.debug("Allocated {}", logMessageBase);
+				return new SingleLogicalSlot(
+					logicalSlotRequestId,
+					physicalSlot,
+					null,
+					Locality.UNKNOWN,
+					this,
+					slotWillBeOccupiedIndefinitely);
+			});
+			logicalSlotFuture.exceptionally(cause -> {
+				LOG.debug("Failed {}", logMessageBase);
+				cancelLogicalSlotRequest(logicalSlotRequestId);
+				return null;
+			});
+			return logicalSlotFuture;
+		}
+
+		void cancelLogicalSlotRequest(ExecutionVertexID executionVertexID) {
+			cancelLogicalSlotRequest(requestedLogicalSlots.getKeyBByKeyA(executionVertexID));
+		}
+
+		void cancelLogicalSlotRequest(SlotRequestId logicalSlotRequestId) {
+			SlotExecutionVertexAssignment assignment = requestedLogicalSlots.getValueByKeyB(logicalSlotRequestId);
+			if (assignment != null) {
+				LOG.debug("Cancel {}", getLogicalSlotString(logicalSlotRequestId, assignment.getExecutionVertexId()));
+				assignment.getLogicalSlotFuture().cancel(false);
+				requestedLogicalSlots.removeKeyB(logicalSlotRequestId);
+			} else {
+				LOG.debug("No SlotExecutionVertexAssignment for logical {} from physical %s", logicalSlotRequestId);
+			}
+			if (requestedLogicalSlots.values().isEmpty()) {
+				sharedSlots.remove(executionSlotSharingGroup);
+				slotProvider.cancelSlotRequest(
+					physicalSlotRequestId,
+					new FlinkException("Slot is being returned from SlotSharingExecutionSlotAllocator."));
+			}
+		}
+
+		@Override
+		public void returnLogicalSlot(LogicalSlot logicalSlot) {
+			cancelLogicalSlotRequest(logicalSlot.getSlotRequestId());
+		}
+
+		@Override
+		public void release(Throwable cause) {
+			for (SlotExecutionVertexAssignment assignment : requestedLogicalSlots.values()) {
+				ExecutionVertexID executionVertexId = assignment.getExecutionVertexId();
+				LOG.debug("Release {}", getLogicalSlotString(requestedLogicalSlots.getKeyBByKeyA(executionVertexId), executionVertexId));
+				assignment

Review comment:
       Maybe add some comments here to explain that all the logical slot futures should have already been completed normally when `release` is invoked? Or even add some sanity checks? It took me some time to reason this assumption.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.SlotProfile;
+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.DualKeyLinkedMap;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Allocates {@link LogicalSlot}s from physical shared slots.
+ *
+ * <p>The allocator maintains a shared slot for each {@link ExecutionSlotSharingGroup}.
+ * It allocates a physical slot for the shared slot and then allocates logical slots from it for scheduled tasks.
+ * The physical slot is lazily allocated for a shared slot, upon any hosted subtask asking for the shared slot.
+ * Each subsequent sharing subtask allocates a logical slot from the existing shared slot. The shared/physical slot
+ * can be released only if all the requested logical slots are released or canceled.
+ */
+class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
+	private static final Logger LOG = LoggerFactory.getLogger(SlotSharingExecutionSlotAllocator.class);
+
+	private final PhysicalSlotProvider slotProvider;
+
+	private final boolean slotWillBeOccupiedIndefinitely;
+
+	private final SlotSharingStrategy slotSharingStrategy;
+
+	private final Map<ExecutionSlotSharingGroup, SharedSlot> sharedSlots;
+
+	private final SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory;
+
+	SlotSharingExecutionSlotAllocator(
+			PhysicalSlotProvider slotProvider,
+			boolean slotWillBeOccupiedIndefinitely,
+			SlotSharingStrategy slotSharingStrategy,
+			SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory) {
+		this.slotProvider = slotProvider;
+		this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+		this.slotSharingStrategy = slotSharingStrategy;
+		this.sharedSlotProfileRetrieverFactory = sharedSlotProfileRetrieverFactory;
+		this.sharedSlots = new HashMap<>();
+	}
+
+	/**
+	 * Creates logical {@link SlotExecutionVertexAssignment}s from physical shared slots.
+	 *
+	 * <p>The allocation has the following steps:
+	 * <ol>
+	 *  <li>Map the executions to {@link ExecutionSlotSharingGroup}s using {@link SlotSharingStrategy}</li>
+	 *  <li>Check which {@link ExecutionSlotSharingGroup}s already have shared slot</li>
+	 *  <li>For all involved {@link ExecutionSlotSharingGroup}s which do not have a shared slot yet:</li>
+	 *  <li>Create a {@link SlotProfile} future using {@link SharedSlotProfileRetriever} and then</li>
+	 *  <li>Allocate a physical slot from the {@link PhysicalSlotProvider}</li>
+	 *  <li>Create a shared slot based on the returned physical slot futures</li>
+	 *  <li>Allocate logical slot futures for the executions from all corresponding shared slots.</li>
+	 *  <li>If a physical slot request fails, associated logical slot requests are canceled within the shared slot</li>
+	 *  <li>Generate {@link SlotExecutionVertexAssignment}s based on the logical slot futures and returns the results.</li>
+	 * </ol>
+	 *
+	 * @param executionVertexSchedulingRequirements the requirements for scheduling the executions.
+	 */
+	@Override
+	public List<SlotExecutionVertexAssignment> allocateSlotsFor(
+			List<ExecutionVertexSchedulingRequirements> executionVertexSchedulingRequirements) {
+		List<ExecutionVertexID> executionVertexIds = executionVertexSchedulingRequirements
+			.stream()
+			.map(ExecutionVertexSchedulingRequirements::getExecutionVertexId)
+			.collect(Collectors.toList());
+
+		SharedSlotProfileRetriever sharedSlotProfileRetriever = sharedSlotProfileRetrieverFactory
+			.createFromBulk(new HashSet<>(executionVertexIds));
+		Map<ExecutionVertexID, SlotExecutionVertexAssignment> assignments = executionVertexIds
+			.stream()
+			.collect(Collectors.groupingBy(slotSharingStrategy::getExecutionSlotSharingGroup))
+			.entrySet()
+			.stream()
+			.flatMap(entry -> allocateLogicalSlotsFromSharedSlot(sharedSlotProfileRetriever, entry.getKey(), entry.getValue()))
+			.collect(Collectors.toMap(SlotExecutionVertexAssignment::getExecutionVertexId, a -> a));
+
+		return executionVertexIds.stream().map(assignments::get).collect(Collectors.toList());
+	}
+
+	@Override
+	public void cancel(ExecutionVertexID executionVertexId) {
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			slotSharingStrategy.getExecutionSlotSharingGroup(executionVertexId);
+		Preconditions.checkNotNull(
+			executionSlotSharingGroup,
+			"There is no ExecutionSlotSharingGroup for ExecutionVertexID " + executionVertexId);
+		SharedSlot slot = sharedSlots.get(executionSlotSharingGroup);
+		if (slot != null) {
+			slot.cancelLogicalSlotRequest(executionVertexId);
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup of ExecutionVertexID {}", executionVertexId);
+		}
+	}
+
+	private Stream<? extends SlotExecutionVertexAssignment> allocateLogicalSlotsFromSharedSlot(
+			SharedSlotProfileRetriever sharedSlotProfileRetriever,
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			Collection<ExecutionVertexID> executions) {
+		SharedSlot sharedSlot = getOrAllocateSharedSlot(executionSlotSharingGroup, sharedSlotProfileRetriever);
+		return executions.stream().map(sharedSlot::createAssignment);
+	}
+
+	private SharedSlot getOrAllocateSharedSlot(
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			SharedSlotProfileRetriever sharedSlotProfileRetriever) {
+		return sharedSlots
+			.computeIfAbsent(executionSlotSharingGroup, group -> {
+				SlotRequestId physicalSlotRequestId = new SlotRequestId();
+				CompletableFuture<PhysicalSlot> physicalSlotFuture = sharedSlotProfileRetriever
+					.getSlotProfileFuture(group)
+					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
+						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
+					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
+				return new SharedSlot(physicalSlotRequestId, group, physicalSlotFuture);
+			});
+	}
+
+	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, SlotExecutionVertexAssignment> 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());
+		}
+
+		SlotExecutionVertexAssignment createAssignment(ExecutionVertexID executionVertexId) {
+			Preconditions.checkArgument(executionSlotSharingGroup.getExecutionVertexIds().contains(executionVertexId));
+			SlotRequestId logicalSlotRequestId = new SlotRequestId();
+			CompletableFuture<LogicalSlot> logicalSlotFuture = allocateLogicalSlot(logicalSlotRequestId, executionVertexId);
+			SlotExecutionVertexAssignment assignment = new SlotExecutionVertexAssignment(executionVertexId, logicalSlotFuture);
+			requestedLogicalSlots.put(
+				executionVertexId,
+				logicalSlotRequestId,
+				assignment);
+			return assignment;
+		}
+
+		private CompletableFuture<LogicalSlot> allocateLogicalSlot(
+				SlotRequestId logicalSlotRequestId,
+				ExecutionVertexID executionVertexId) {
+			String logMessageBase = getLogicalSlotString(logicalSlotRequestId, executionVertexId);
+			LOG.debug("Request a {}", logMessageBase);
+			CompletableFuture<LogicalSlot> logicalSlotFuture = slotContextFuture.thenApply(physicalSlot -> {
+				LOG.debug("Allocated {}", logMessageBase);
+				return new SingleLogicalSlot(
+					logicalSlotRequestId,
+					physicalSlot,
+					null,
+					Locality.UNKNOWN,
+					this,
+					slotWillBeOccupiedIndefinitely);
+			});
+			logicalSlotFuture.exceptionally(cause -> {
+				LOG.debug("Failed {}", logMessageBase);
+				cancelLogicalSlotRequest(logicalSlotRequestId);
+				return null;
+			});
+			return logicalSlotFuture;
+		}
+
+		void cancelLogicalSlotRequest(ExecutionVertexID executionVertexID) {
+			cancelLogicalSlotRequest(requestedLogicalSlots.getKeyBByKeyA(executionVertexID));
+		}
+
+		void cancelLogicalSlotRequest(SlotRequestId logicalSlotRequestId) {
+			SlotExecutionVertexAssignment assignment = requestedLogicalSlots.getValueByKeyB(logicalSlotRequestId);
+			if (assignment != null) {
+				LOG.debug("Cancel {}", getLogicalSlotString(logicalSlotRequestId, assignment.getExecutionVertexId()));
+				assignment.getLogicalSlotFuture().cancel(false);
+				requestedLogicalSlots.removeKeyB(logicalSlotRequestId);
+			} else {
+				LOG.debug("No SlotExecutionVertexAssignment for logical {} from physical %s", logicalSlotRequestId);
+			}
+			if (requestedLogicalSlots.values().isEmpty()) {
+				sharedSlots.remove(executionSlotSharingGroup);
+				slotProvider.cancelSlotRequest(
+					physicalSlotRequestId,
+					new FlinkException("Slot is being returned from SlotSharingExecutionSlotAllocator."));
+			}
+		}
+
+		@Override
+		public void returnLogicalSlot(LogicalSlot logicalSlot) {
+			cancelLogicalSlotRequest(logicalSlot.getSlotRequestId());
+		}
+
+		@Override
+		public void release(Throwable cause) {
+			for (SlotExecutionVertexAssignment assignment : requestedLogicalSlots.values()) {
+				ExecutionVertexID executionVertexId = assignment.getExecutionVertexId();
+				LOG.debug("Release {}", getLogicalSlotString(requestedLogicalSlots.getKeyBByKeyA(executionVertexId), executionVertexId));
+				assignment
+					.getLogicalSlotFuture()
+					.thenAccept(logicalSlot -> ((PhysicalSlot.Payload) logicalSlot).release(cause));
+			}
+		}
+
+		@Override
+		public boolean willOccupySlotIndefinitely() {
+			return slotWillBeOccupiedIndefinitely;
+		}
+
+		private String getLogicalSlotString(SlotRequestId logicalSlotRequestId, ExecutionVertexID executionVertexId) {
+			return String.format(
+				"logical %s for execution vertex (id %s) from the physical %s",

Review comment:
       ```suggestion
   				"logical slot %s for execution vertex (id %s) from the physical slot %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] flinkbot edited a comment on pull request #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8a735691f03d9abd842d7917303ccf9c9fe21e56 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212) 
   
   <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 closed pull request #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   


----------------------------------------------------------------
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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8a735691f03d9abd842d7917303ccf9c9fe21e56 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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java
##########
@@ -0,0 +1,575 @@
+/*
+ * 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.clusterframework.types.SlotProfile;
+import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway;
+import org.apache.flink.runtime.instance.SimpleSlotContext;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.TestingPayload;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import org.junit.Test;
+import org.powermock.core.IdentityHashSet;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.BiConsumer;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Test suite for {@link SlotSharingExecutionSlotAllocator}.
+ */
+public class SlotSharingExecutionSlotAllocatorTest {
+	@Test
+	public void testSlotProfileRequestAskedBulkAndGroup() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		context.allocateSlotsFor(0, 2);
+
+		Set<ExecutionVertexID> ids = new HashSet<>(context.getReqIds(0, 2));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedBulks(), containsInAnyOrder(ids));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedGroups(), containsInAnyOrder(executionSlotSharingGroup));
+	}
+
+	@Test
+	public void testSlotRequestCompletionAfterProfileCompletion() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).completeSlotProfileFutureManually().build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 2);
+
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(false)));
+		context.getSlotProfileRetrieverFactory().completeSlotProfileFutureFor(executionSlotSharingGroup);
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(true)));
+	}
+
+	@Test
+	public void testSlotRequestProfile() {
+		ResourceProfile physicalsSlotResourceProfile = ResourceProfile.fromResources(3, 5);
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+		context.getSlotProfileRetrieverFactory().addGroupResourceProfile(executionSlotSharingGroup, physicalsSlotResourceProfile);
+
+		context.allocateSlotsFor(0, 2);
+
+		Optional<PhysicalSlotRequest> slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst();
+		assertThat(slotRequest.isPresent(), is(true));
+		slotRequest.ifPresent(r -> assertThat(r.getSlotProfile().getPhysicalSlotResourceProfile(), is(physicalsSlotResourceProfile)));
+	}
+
+	@Test
+	public void testNewAllocatePhysicalSlotForSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2, 2).build();
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 4);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(0, 4).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testAllocateLogicalSlotFromAvailableSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+
+		context.allocateSlotsFor(0, 1);
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(1, 2);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		// execution 0 from the first allocateSlotsFor call and execution 1 from the second allocateSlotsFor call
+		// share a slot, therefore only one physical slot allocation should happen
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(1, 2).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+	}
+
+	@Test
+	public void testDuplicateAllocationDoesNotRecreateLogicalSlotFuture() throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(1).build();
+
+		SlotExecutionVertexAssignment assignment1 = context.allocateSlotsFor(0, 1).get(0);
+		SlotExecutionVertexAssignment assignment2 = context.allocateSlotsFor(0, 1).get(0);
+
+		assertThat(assignment1.getLogicalSlotFuture().get() == assignment2.getLogicalSlotFuture().get(), is(true));
+	}
+
+	@Test
+	public void testFailedPhysicalSlotRequestFailsLogicalSlotFuturesAndRemovesSharedSlot() {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.completePhysicalSlotFutureManually()
+			.build();
+		CompletableFuture<LogicalSlot> logicalSlotFuture = context.allocateSlotsFor(0, 1).get(0).getLogicalSlotFuture();
+		SlotRequestId slotRequestId = context.getSlotProvider().getRequests().values().stream().findFirst().get().getSlotRequestId();
+
+		assertThat(logicalSlotFuture.isDone(), is(false));
+		context.getSlotProvider().failPhysicalSlotFutureFor(slotRequestId, new Throwable());
+		assertThat(logicalSlotFuture.isCompletedExceptionally(), is(true));
+
+		// next allocation allocates new shared slot
+		context.allocateSlotsFor(0, 1);
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyFalse() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(false);
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyTrue() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(true);
+	}
+
+	private static void testSlotWillBeOccupiedIndefinitely(boolean slotWillBeOccupiedIndefinitely) throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.setSlotWillBeOccupiedIndefinitely(slotWillBeOccupiedIndefinitely)
+			.build();
+		context.allocateSlotsFor(0, 1);
+
+		PhysicalSlotRequest slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst().get();
+		assertThat(slotRequest.willSlotBeOccupiedIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+
+		TestingPhysicalSlot physicalSlot = context.getSlotProvider().getResponses().get(slotRequest.getSlotRequestId()).get();
+		assertThat(physicalSlot.getPayload(), notNullValue());
+		assertThat(physicalSlot.getPayload().willOccupySlotIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+	}
+
+	@Test
+	public void testReturningLogicalSlotsRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			false,
+			(context, assignment) -> {
+				try {
+					assignment.getLogicalSlotFuture().get().releaseSlot(null);
+				} catch (InterruptedException | ExecutionException e) {
+					throw new FlinkRuntimeException("Unexpected", e);
+				}
+			});
+	}
+
+	@Test
+	public void testLogicalSlotCancelsPhysicalSlotRequestAndRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			true,
+			(context, assignment) -> {
+				context.getAllocator().cancel(assignment.getExecutionVertexId());
+				try {
+					assignment.getLogicalSlotFuture().get();
+					fail("THe logical future must finish with the cancellation exception");
+				} catch (InterruptedException | ExecutionException e) {
+					assertThat(e.getCause(), instanceOf(CancellationException.class));
+				}
+			});
+	}
+
+	private static void testLogicalSlotRequestCancellation(
+			boolean completePhysicalSlotFutureManually,
+			BiConsumer<AllocationContext, SlotExecutionVertexAssignment> cancelAction) {
+		//if (completePhysicalSlotRequest) {

Review comment:
       legacy comment?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java
##########
@@ -0,0 +1,575 @@
+/*
+ * 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.clusterframework.types.SlotProfile;
+import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway;
+import org.apache.flink.runtime.instance.SimpleSlotContext;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.TestingPayload;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import org.junit.Test;
+import org.powermock.core.IdentityHashSet;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.BiConsumer;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Test suite for {@link SlotSharingExecutionSlotAllocator}.
+ */
+public class SlotSharingExecutionSlotAllocatorTest {
+	@Test
+	public void testSlotProfileRequestAskedBulkAndGroup() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		context.allocateSlotsFor(0, 2);
+
+		Set<ExecutionVertexID> ids = new HashSet<>(context.getReqIds(0, 2));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedBulks(), containsInAnyOrder(ids));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedGroups(), containsInAnyOrder(executionSlotSharingGroup));
+	}
+
+	@Test
+	public void testSlotRequestCompletionAfterProfileCompletion() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).completeSlotProfileFutureManually().build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 2);
+
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(false)));
+		context.getSlotProfileRetrieverFactory().completeSlotProfileFutureFor(executionSlotSharingGroup);
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(true)));
+	}
+
+	@Test
+	public void testSlotRequestProfile() {
+		ResourceProfile physicalsSlotResourceProfile = ResourceProfile.fromResources(3, 5);
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+		context.getSlotProfileRetrieverFactory().addGroupResourceProfile(executionSlotSharingGroup, physicalsSlotResourceProfile);
+
+		context.allocateSlotsFor(0, 2);
+
+		Optional<PhysicalSlotRequest> slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst();
+		assertThat(slotRequest.isPresent(), is(true));
+		slotRequest.ifPresent(r -> assertThat(r.getSlotProfile().getPhysicalSlotResourceProfile(), is(physicalsSlotResourceProfile)));
+	}
+
+	@Test
+	public void testNewAllocatePhysicalSlotForSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2, 2).build();
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 4);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(0, 4).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testAllocateLogicalSlotFromAvailableSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+
+		context.allocateSlotsFor(0, 1);
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(1, 2);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		// execution 0 from the first allocateSlotsFor call and execution 1 from the second allocateSlotsFor call
+		// share a slot, therefore only one physical slot allocation should happen
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(1, 2).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+	}
+
+	@Test
+	public void testDuplicateAllocationDoesNotRecreateLogicalSlotFuture() throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(1).build();
+
+		SlotExecutionVertexAssignment assignment1 = context.allocateSlotsFor(0, 1).get(0);
+		SlotExecutionVertexAssignment assignment2 = context.allocateSlotsFor(0, 1).get(0);
+
+		assertThat(assignment1.getLogicalSlotFuture().get() == assignment2.getLogicalSlotFuture().get(), is(true));
+	}
+
+	@Test
+	public void testFailedPhysicalSlotRequestFailsLogicalSlotFuturesAndRemovesSharedSlot() {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.completePhysicalSlotFutureManually()
+			.build();
+		CompletableFuture<LogicalSlot> logicalSlotFuture = context.allocateSlotsFor(0, 1).get(0).getLogicalSlotFuture();
+		SlotRequestId slotRequestId = context.getSlotProvider().getRequests().values().stream().findFirst().get().getSlotRequestId();
+
+		assertThat(logicalSlotFuture.isDone(), is(false));
+		context.getSlotProvider().failPhysicalSlotFutureFor(slotRequestId, new Throwable());
+		assertThat(logicalSlotFuture.isCompletedExceptionally(), is(true));
+
+		// next allocation allocates new shared slot
+		context.allocateSlotsFor(0, 1);
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyFalse() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(false);
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyTrue() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(true);
+	}
+
+	private static void testSlotWillBeOccupiedIndefinitely(boolean slotWillBeOccupiedIndefinitely) throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.setSlotWillBeOccupiedIndefinitely(slotWillBeOccupiedIndefinitely)
+			.build();
+		context.allocateSlotsFor(0, 1);
+
+		PhysicalSlotRequest slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst().get();
+		assertThat(slotRequest.willSlotBeOccupiedIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+
+		TestingPhysicalSlot physicalSlot = context.getSlotProvider().getResponses().get(slotRequest.getSlotRequestId()).get();
+		assertThat(physicalSlot.getPayload(), notNullValue());
+		assertThat(physicalSlot.getPayload().willOccupySlotIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+	}
+
+	@Test
+	public void testReturningLogicalSlotsRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			false,
+			(context, assignment) -> {
+				try {
+					assignment.getLogicalSlotFuture().get().releaseSlot(null);
+				} catch (InterruptedException | ExecutionException e) {
+					throw new FlinkRuntimeException("Unexpected", e);
+				}
+			});
+	}
+
+	@Test
+	public void testLogicalSlotCancelsPhysicalSlotRequestAndRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			true,
+			(context, assignment) -> {
+				context.getAllocator().cancel(assignment.getExecutionVertexId());
+				try {
+					assignment.getLogicalSlotFuture().get();
+					fail("THe logical future must finish with the cancellation exception");

Review comment:
       typo: `THe` -> `The `

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java
##########
@@ -0,0 +1,575 @@
+/*
+ * 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.clusterframework.types.SlotProfile;
+import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway;
+import org.apache.flink.runtime.instance.SimpleSlotContext;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.TestingPayload;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import org.junit.Test;
+import org.powermock.core.IdentityHashSet;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.BiConsumer;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Test suite for {@link SlotSharingExecutionSlotAllocator}.
+ */
+public class SlotSharingExecutionSlotAllocatorTest {
+	@Test
+	public void testSlotProfileRequestAskedBulkAndGroup() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		context.allocateSlotsFor(0, 2);
+
+		Set<ExecutionVertexID> ids = new HashSet<>(context.getReqIds(0, 2));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedBulks(), containsInAnyOrder(ids));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedGroups(), containsInAnyOrder(executionSlotSharingGroup));
+	}
+
+	@Test
+	public void testSlotRequestCompletionAfterProfileCompletion() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).completeSlotProfileFutureManually().build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 2);
+
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(false)));
+		context.getSlotProfileRetrieverFactory().completeSlotProfileFutureFor(executionSlotSharingGroup);
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(true)));
+	}
+
+	@Test
+	public void testSlotRequestProfile() {
+		ResourceProfile physicalsSlotResourceProfile = ResourceProfile.fromResources(3, 5);
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+		context.getSlotProfileRetrieverFactory().addGroupResourceProfile(executionSlotSharingGroup, physicalsSlotResourceProfile);
+
+		context.allocateSlotsFor(0, 2);
+
+		Optional<PhysicalSlotRequest> slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst();
+		assertThat(slotRequest.isPresent(), is(true));
+		slotRequest.ifPresent(r -> assertThat(r.getSlotProfile().getPhysicalSlotResourceProfile(), is(physicalsSlotResourceProfile)));
+	}
+
+	@Test
+	public void testNewAllocatePhysicalSlotForSharedSlot() {

Review comment:
       do you mean `testAllocatePhysicalSlotForNewSharedSlot `?




----------------------------------------------------------------
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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07fd40233580c1641fda58742e015144d767c2fe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07fd40233580c1641fda58742e015144d767c2fe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5256",
       "triggerID" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5276",
       "triggerID" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5290",
       "triggerID" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5304",
       "triggerID" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e98d6b28318bd322f7114bab3b1fdd753936362",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5439",
       "triggerID" : "5e98d6b28318bd322f7114bab3b1fdd753936362",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f29ba5bcfd9ee2632ae34f66e635074858c9b37b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5487",
       "triggerID" : "f29ba5bcfd9ee2632ae34f66e635074858c9b37b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 07fd40233580c1641fda58742e015144d767c2fe UNKNOWN
   * 5e98d6b28318bd322f7114bab3b1fdd753936362 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5439) 
   * f29ba5bcfd9ee2632ae34f66e635074858c9b37b Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5487) 
   
   <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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.SlotProfile;
+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.DualKeyLinkedMap;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Allocates {@link LogicalSlot}s from physical shared slots.
+ *
+ * <p>The allocator maintains a shared slot for each {@link ExecutionSlotSharingGroup}.
+ * It allocates a physical slot for the shared slot and then allocates logical slots from it for scheduled tasks.
+ * The physical slot is lazily allocated for a shared slot, upon any hosted subtask asking for the shared slot.
+ * Each subsequent sharing subtask allocates a logical slot from the existing shared slot. The shared/physical slot
+ * can be released only if all the requested logical slots are released or canceled.
+ */
+class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
+	private static final Logger LOG = LoggerFactory.getLogger(SlotSharingExecutionSlotAllocator.class);
+
+	private final PhysicalSlotProvider slotProvider;
+
+	private final boolean slotWillBeOccupiedIndefinitely;
+
+	private final SlotSharingStrategy slotSharingStrategy;
+
+	private final Map<ExecutionSlotSharingGroup, SharedSlot> sharedSlots;
+
+	private final SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory;
+
+	SlotSharingExecutionSlotAllocator(
+			PhysicalSlotProvider slotProvider,
+			boolean slotWillBeOccupiedIndefinitely,
+			SlotSharingStrategy slotSharingStrategy,
+			SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory) {
+		this.slotProvider = slotProvider;
+		this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+		this.slotSharingStrategy = slotSharingStrategy;
+		this.sharedSlotProfileRetrieverFactory = sharedSlotProfileRetrieverFactory;
+		this.sharedSlots = new HashMap<>();
+	}
+
+	/**
+	 * Creates logical {@link SlotExecutionVertexAssignment}s from physical shared slots.
+	 *
+	 * <p>The allocation has the following steps:
+	 * <ol>
+	 *  <li>Map the executions to {@link ExecutionSlotSharingGroup}s using {@link SlotSharingStrategy}</li>
+	 *  <li>Check which {@link ExecutionSlotSharingGroup}s already have shared slot</li>
+	 *  <li>For all involved {@link ExecutionSlotSharingGroup}s which do not have a shared slot yet:</li>
+	 *  <li>Create a {@link SlotProfile} future using {@link SharedSlotProfileRetriever} and then</li>
+	 *  <li>Allocate a physical slot from the {@link PhysicalSlotProvider}</li>
+	 *  <li>Create a shared slot based on the returned physical slot futures</li>
+	 *  <li>Allocate logical slot futures for the executions from all corresponding shared slots.</li>
+	 *  <li>If a physical slot request fails, associated logical slot requests are canceled within the shared slot</li>
+	 *  <li>Generate {@link SlotExecutionVertexAssignment}s based on the logical slot futures and returns the results.</li>
+	 * </ol>
+	 *
+	 * @param executionVertexSchedulingRequirements the requirements for scheduling the executions.
+	 */
+	@Override
+	public List<SlotExecutionVertexAssignment> allocateSlotsFor(
+			List<ExecutionVertexSchedulingRequirements> executionVertexSchedulingRequirements) {
+		List<ExecutionVertexID> executionVertexIds = executionVertexSchedulingRequirements
+			.stream()
+			.map(ExecutionVertexSchedulingRequirements::getExecutionVertexId)
+			.collect(Collectors.toList());
+
+		SharedSlotProfileRetriever sharedSlotProfileRetriever = sharedSlotProfileRetrieverFactory
+			.createFromBulk(new HashSet<>(executionVertexIds));
+		Map<ExecutionVertexID, SlotExecutionVertexAssignment> assignments = executionVertexIds
+			.stream()
+			.collect(Collectors.groupingBy(slotSharingStrategy::getExecutionSlotSharingGroup))
+			.entrySet()
+			.stream()
+			.flatMap(entry -> allocateLogicalSlotsFromSharedSlot(sharedSlotProfileRetriever, entry.getKey(), entry.getValue()))
+			.collect(Collectors.toMap(SlotExecutionVertexAssignment::getExecutionVertexId, a -> a));
+
+		return executionVertexIds.stream().map(assignments::get).collect(Collectors.toList());
+	}
+
+	@Override
+	public void cancel(ExecutionVertexID executionVertexId) {
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			slotSharingStrategy.getExecutionSlotSharingGroup(executionVertexId);
+		Preconditions.checkNotNull(
+			executionSlotSharingGroup,
+			"There is no ExecutionSlotSharingGroup for ExecutionVertexID " + executionVertexId);
+		SharedSlot slot = sharedSlots.get(executionSlotSharingGroup);
+		if (slot != null) {
+			slot.cancelLogicalSlotRequest(executionVertexId);
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup of ExecutionVertexID {}", executionVertexId);
+		}
+	}
+
+	private Stream<? extends SlotExecutionVertexAssignment> allocateLogicalSlotsFromSharedSlot(
+			SharedSlotProfileRetriever sharedSlotProfileRetriever,
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			Collection<ExecutionVertexID> executions) {
+		SharedSlot sharedSlot = getOrAllocateSharedSlot(executionSlotSharingGroup, sharedSlotProfileRetriever);
+		return executions.stream().map(sharedSlot::createAssignment);
+	}
+
+	private SharedSlot getOrAllocateSharedSlot(
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			SharedSlotProfileRetriever sharedSlotProfileRetriever) {
+		return sharedSlots
+			.computeIfAbsent(executionSlotSharingGroup, group -> {
+				SlotRequestId physicalSlotRequestId = new SlotRequestId();
+				CompletableFuture<PhysicalSlot> physicalSlotFuture = sharedSlotProfileRetriever
+					.getSlotProfileFuture(group)
+					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
+						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
+					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
+				return new SharedSlot(physicalSlotRequestId, group, physicalSlotFuture);
+			});
+	}
+
+	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, SlotExecutionVertexAssignment> 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());
+		}
+
+		SlotExecutionVertexAssignment createAssignment(ExecutionVertexID executionVertexId) {
+			Preconditions.checkArgument(executionSlotSharingGroup.getExecutionVertexIds().contains(executionVertexId));
+			SlotRequestId logicalSlotRequestId = new SlotRequestId();
+			CompletableFuture<LogicalSlot> logicalSlotFuture = allocateLogicalSlot(logicalSlotRequestId, executionVertexId);
+			SlotExecutionVertexAssignment assignment = new SlotExecutionVertexAssignment(executionVertexId, logicalSlotFuture);
+			requestedLogicalSlots.put(
+				executionVertexId,
+				logicalSlotRequestId,
+				assignment);
+			return assignment;
+		}
+
+		private CompletableFuture<LogicalSlot> allocateLogicalSlot(
+				SlotRequestId logicalSlotRequestId,
+				ExecutionVertexID executionVertexId) {
+			String logMessageBase = getLogicalSlotString(logicalSlotRequestId, executionVertexId);

Review comment:
       I will change the order:
   ```
   requestedLogicalSlots.put(executionVertexId, logicalSlotRequestId, logicalSlotFuture);
   logicalSlotFuture.exceptionally(cause -> {
   	LOG.debug("Failed {}", logMessageBase);
   	cancelLogicalSlotRequest(logicalSlotRequestId);
   	return null;
   });			
   ```
   In this case, if `slotContextFuture` has already completed exceptionally, it will fail the `logicalSlotFuture` immediately and `logicalSlotFuture.exceptionally -> cancelLogicalSlotRequest` should do the cleanup.




----------------------------------------------------------------
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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.SlotProfile;
+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.DualKeyLinkedMap;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Allocates {@link LogicalSlot}s from physical shared slots.
+ *
+ * <p>The allocator maintains a shared slot for each {@link ExecutionSlotSharingGroup}.
+ * It allocates a physical slot for the shared slot and then allocates logical slots from it for scheduled tasks.
+ * The physical slot is lazily allocated for a shared slot, upon any hosted subtask asking for the shared slot.
+ * Each subsequent sharing subtask allocates a logical slot from the existing shared slot. The shared/physical slot
+ * can be released only if all the requested logical slots are released or canceled.
+ */
+class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
+	private static final Logger LOG = LoggerFactory.getLogger(SlotSharingExecutionSlotAllocator.class);
+
+	private final PhysicalSlotProvider slotProvider;
+
+	private final boolean slotWillBeOccupiedIndefinitely;
+
+	private final SlotSharingStrategy slotSharingStrategy;
+
+	private final Map<ExecutionSlotSharingGroup, SharedSlot> sharedSlots;
+
+	private final SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory;
+
+	SlotSharingExecutionSlotAllocator(
+			PhysicalSlotProvider slotProvider,
+			boolean slotWillBeOccupiedIndefinitely,
+			SlotSharingStrategy slotSharingStrategy,
+			SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory) {
+		this.slotProvider = slotProvider;
+		this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+		this.slotSharingStrategy = slotSharingStrategy;
+		this.sharedSlotProfileRetrieverFactory = sharedSlotProfileRetrieverFactory;
+		this.sharedSlots = new HashMap<>();
+	}
+
+	/**
+	 * Creates logical {@link SlotExecutionVertexAssignment}s from physical shared slots.
+	 *
+	 * <p>The allocation has the following steps:
+	 * <ol>
+	 *  <li>Map the executions to {@link ExecutionSlotSharingGroup}s using {@link SlotSharingStrategy}</li>
+	 *  <li>Check which {@link ExecutionSlotSharingGroup}s already have shared slot</li>
+	 *  <li>For all involved {@link ExecutionSlotSharingGroup}s which do not have a shared slot yet:</li>
+	 *  <li>Create a {@link SlotProfile} future using {@link SharedSlotProfileRetriever} and then</li>
+	 *  <li>Allocate a physical slot from the {@link PhysicalSlotProvider}</li>
+	 *  <li>Create a shared slot based on the returned physical slot futures</li>
+	 *  <li>Allocate logical slot futures for the executions from all corresponding shared slots.</li>
+	 *  <li>If a physical slot request fails, associated logical slot requests are canceled within the shared slot</li>
+	 *  <li>Generate {@link SlotExecutionVertexAssignment}s based on the logical slot futures and returns the results.</li>
+	 * </ol>
+	 *
+	 * @param executionVertexSchedulingRequirements the requirements for scheduling the executions.
+	 */
+	@Override
+	public List<SlotExecutionVertexAssignment> allocateSlotsFor(
+			List<ExecutionVertexSchedulingRequirements> executionVertexSchedulingRequirements) {
+		List<ExecutionVertexID> executionVertexIds = executionVertexSchedulingRequirements
+			.stream()
+			.map(ExecutionVertexSchedulingRequirements::getExecutionVertexId)
+			.collect(Collectors.toList());
+
+		SharedSlotProfileRetriever sharedSlotProfileRetriever = sharedSlotProfileRetrieverFactory
+			.createFromBulk(new HashSet<>(executionVertexIds));
+		Map<ExecutionVertexID, SlotExecutionVertexAssignment> assignments = executionVertexIds
+			.stream()
+			.collect(Collectors.groupingBy(slotSharingStrategy::getExecutionSlotSharingGroup))
+			.entrySet()
+			.stream()
+			.flatMap(entry -> allocateLogicalSlotsFromSharedSlot(sharedSlotProfileRetriever, entry.getKey(), entry.getValue()))
+			.collect(Collectors.toMap(SlotExecutionVertexAssignment::getExecutionVertexId, a -> a));
+
+		return executionVertexIds.stream().map(assignments::get).collect(Collectors.toList());
+	}
+
+	@Override
+	public void cancel(ExecutionVertexID executionVertexId) {
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			slotSharingStrategy.getExecutionSlotSharingGroup(executionVertexId);
+		Preconditions.checkNotNull(
+			executionSlotSharingGroup,
+			"There is no ExecutionSlotSharingGroup for ExecutionVertexID " + executionVertexId);
+		SharedSlot slot = sharedSlots.get(executionSlotSharingGroup);
+		if (slot != null) {
+			slot.cancelLogicalSlotRequest(executionVertexId);
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup of ExecutionVertexID {}", executionVertexId);
+		}
+	}
+
+	private Stream<? extends SlotExecutionVertexAssignment> allocateLogicalSlotsFromSharedSlot(
+			SharedSlotProfileRetriever sharedSlotProfileRetriever,
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			Collection<ExecutionVertexID> executions) {
+		SharedSlot sharedSlot = getOrAllocateSharedSlot(executionSlotSharingGroup, sharedSlotProfileRetriever);
+		return executions.stream().map(sharedSlot::createAssignment);
+	}
+
+	private SharedSlot getOrAllocateSharedSlot(
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			SharedSlotProfileRetriever sharedSlotProfileRetriever) {
+		return sharedSlots
+			.computeIfAbsent(executionSlotSharingGroup, group -> {
+				SlotRequestId physicalSlotRequestId = new SlotRequestId();
+				CompletableFuture<PhysicalSlot> physicalSlotFuture = sharedSlotProfileRetriever
+					.getSlotProfileFuture(group)
+					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
+						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
+					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
+				return new SharedSlot(physicalSlotRequestId, group, physicalSlotFuture);
+			});
+	}
+
+	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, SlotExecutionVertexAssignment> 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());
+		}
+
+		SlotExecutionVertexAssignment createAssignment(ExecutionVertexID executionVertexId) {
+			Preconditions.checkArgument(executionSlotSharingGroup.getExecutionVertexIds().contains(executionVertexId));
+			SlotRequestId logicalSlotRequestId = new SlotRequestId();
+			CompletableFuture<LogicalSlot> logicalSlotFuture = allocateLogicalSlot(logicalSlotRequestId, executionVertexId);
+			SlotExecutionVertexAssignment assignment = new SlotExecutionVertexAssignment(executionVertexId, logicalSlotFuture);
+			requestedLogicalSlots.put(
+				executionVertexId,
+				logicalSlotRequestId,
+				assignment);
+			return assignment;
+		}
+
+		private CompletableFuture<LogicalSlot> allocateLogicalSlot(
+				SlotRequestId logicalSlotRequestId,
+				ExecutionVertexID executionVertexId) {
+			String logMessageBase = getLogicalSlotString(logicalSlotRequestId, executionVertexId);
+			LOG.debug("Request a {}", logMessageBase);
+			CompletableFuture<LogicalSlot> logicalSlotFuture = slotContextFuture.thenApply(physicalSlot -> {
+				LOG.debug("Allocated {}", logMessageBase);
+				return new SingleLogicalSlot(
+					logicalSlotRequestId,
+					physicalSlot,
+					null,
+					Locality.UNKNOWN,
+					this,
+					slotWillBeOccupiedIndefinitely);
+			});
+			logicalSlotFuture.exceptionally(cause -> {
+				LOG.debug("Failed {}", logMessageBase);
+				cancelLogicalSlotRequest(logicalSlotRequestId);
+				return null;
+			});
+			return logicalSlotFuture;
+		}
+
+		void cancelLogicalSlotRequest(ExecutionVertexID executionVertexID) {
+			cancelLogicalSlotRequest(requestedLogicalSlots.getKeyBByKeyA(executionVertexID));
+		}
+
+		void cancelLogicalSlotRequest(SlotRequestId logicalSlotRequestId) {
+			SlotExecutionVertexAssignment assignment = requestedLogicalSlots.getValueByKeyB(logicalSlotRequestId);
+			if (assignment != null) {
+				LOG.debug("Cancel {}", getLogicalSlotString(logicalSlotRequestId, assignment.getExecutionVertexId()));
+				assignment.getLogicalSlotFuture().cancel(false);
+				requestedLogicalSlots.removeKeyB(logicalSlotRequestId);
+			} else {
+				LOG.debug("No SlotExecutionVertexAssignment for logical {} from physical %s", logicalSlotRequestId);
+			}
+			if (requestedLogicalSlots.values().isEmpty()) {
+				sharedSlots.remove(executionSlotSharingGroup);
+				slotProvider.cancelSlotRequest(
+					physicalSlotRequestId,
+					new FlinkException("Slot is being returned from SlotSharingExecutionSlotAllocator."));
+			}
+		}
+
+		@Override
+		public void returnLogicalSlot(LogicalSlot logicalSlot) {
+			cancelLogicalSlotRequest(logicalSlot.getSlotRequestId());
+		}
+
+		@Override
+		public void release(Throwable cause) {
+			for (SlotExecutionVertexAssignment assignment : requestedLogicalSlots.values()) {
+				ExecutionVertexID executionVertexId = assignment.getExecutionVertexId();
+				LOG.debug("Release {}", getLogicalSlotString(requestedLogicalSlots.getKeyBByKeyA(executionVertexId), executionVertexId));
+				assignment

Review comment:
       Do they have to complete normally?
   The method does not look to assume this. It releases the logical slot only if its future is done:
   `.thenAccept(logicalSlot -> logicalSlot.release(cause));`
   I thought that `logicalSlot.release` will eventually trigger `returnLogicalSlot/cancelLogicalSlotRequest`




----------------------------------------------------------------
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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07fd40233580c1641fda58742e015144d767c2fe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07fd40233580c1641fda58742e015144d767c2fe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5256",
       "triggerID" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5276",
       "triggerID" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5290",
       "triggerID" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5304",
       "triggerID" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e98d6b28318bd322f7114bab3b1fdd753936362",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5439",
       "triggerID" : "5e98d6b28318bd322f7114bab3b1fdd753936362",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 07fd40233580c1641fda58742e015144d767c2fe UNKNOWN
   * 5e98d6b28318bd322f7114bab3b1fdd753936362 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5439) 
   
   <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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07fd40233580c1641fda58742e015144d767c2fe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07fd40233580c1641fda58742e015144d767c2fe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5256",
       "triggerID" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5276",
       "triggerID" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5290",
       "triggerID" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5304",
       "triggerID" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e98d6b28318bd322f7114bab3b1fdd753936362",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5439",
       "triggerID" : "5e98d6b28318bd322f7114bab3b1fdd753936362",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f29ba5bcfd9ee2632ae34f66e635074858c9b37b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5487",
       "triggerID" : "f29ba5bcfd9ee2632ae34f66e635074858c9b37b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ba4f4388232fbb26ce7fe440915f07c7b79f2b15",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5509",
       "triggerID" : "ba4f4388232fbb26ce7fe440915f07c7b79f2b15",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 07fd40233580c1641fda58742e015144d767c2fe UNKNOWN
   * f29ba5bcfd9ee2632ae34f66e635074858c9b37b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5487) 
   * ba4f4388232fbb26ce7fe440915f07c7b79f2b15 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5509) 
   
   <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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07fd40233580c1641fda58742e015144d767c2fe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07fd40233580c1641fda58742e015144d767c2fe",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 223e7673843cdc4fc6467ce0ed9ad0ff0630aef5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251) 
   * 07fd40233580c1641fda58742e015144d767c2fe 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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 708450b388222f8c266604429e604327bce83491 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242) 
   * 223e7673843cdc4fc6467ce0ed9ad0ff0630aef5 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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07fd40233580c1641fda58742e015144d767c2fe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07fd40233580c1641fda58742e015144d767c2fe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5256",
       "triggerID" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 07fd40233580c1641fda58742e015144d767c2fe UNKNOWN
   * 2d0c93809d722ca1325ebec8aa4de38c83cc3f5f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5256) 
   
   <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 removed a comment on pull request #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

Posted by GitBox <gi...@apache.org>.
azagrebin removed a comment on pull request #13071:
URL: https://github.com/apache/flink/pull/13071#issuecomment-673311395


   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] zhuzhurk commented on a change in pull request #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java
##########
@@ -0,0 +1,575 @@
+/*
+ * 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.clusterframework.types.SlotProfile;
+import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway;
+import org.apache.flink.runtime.instance.SimpleSlotContext;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.TestingPayload;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import org.junit.Test;
+import org.powermock.core.IdentityHashSet;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.BiConsumer;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Test suite for {@link SlotSharingExecutionSlotAllocator}.
+ */
+public class SlotSharingExecutionSlotAllocatorTest {
+	@Test
+	public void testSlotProfileRequestAskedBulkAndGroup() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		context.allocateSlotsFor(0, 2);
+
+		Set<ExecutionVertexID> ids = new HashSet<>(context.getReqIds(0, 2));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedBulks(), containsInAnyOrder(ids));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedGroups(), containsInAnyOrder(executionSlotSharingGroup));
+	}
+
+	@Test
+	public void testSlotRequestCompletionAfterProfileCompletion() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).completeSlotProfileFutureManually().build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 2);
+
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(false)));
+		context.getSlotProfileRetrieverFactory().completeSlotProfileFutureFor(executionSlotSharingGroup);
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(true)));
+	}
+
+	@Test
+	public void testSlotRequestProfile() {
+		ResourceProfile physicalsSlotResourceProfile = ResourceProfile.fromResources(3, 5);
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+		context.getSlotProfileRetrieverFactory().addGroupResourceProfile(executionSlotSharingGroup, physicalsSlotResourceProfile);
+
+		context.allocateSlotsFor(0, 2);
+
+		Optional<PhysicalSlotRequest> slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst();
+		assertThat(slotRequest.isPresent(), is(true));
+		slotRequest.ifPresent(r -> assertThat(r.getSlotProfile().getPhysicalSlotResourceProfile(), is(physicalsSlotResourceProfile)));
+	}
+
+	@Test
+	public void testNewAllocatePhysicalSlotForSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2, 2).build();
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 4);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(0, 4).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testAllocateLogicalSlotFromAvailableSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+
+		context.allocateSlotsFor(0, 1);
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(1, 2);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		// execution 0 from the first allocateSlotsFor call and execution 1 from the second allocateSlotsFor call
+		// share a slot, therefore only one physical slot allocation should happen
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(1, 2).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+	}
+
+	@Test
+	public void testDuplicateAllocationDoesNotRecreateLogicalSlotFuture() throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(1).build();
+
+		SlotExecutionVertexAssignment assignment1 = context.allocateSlotsFor(0, 1).get(0);
+		SlotExecutionVertexAssignment assignment2 = context.allocateSlotsFor(0, 1).get(0);
+
+		assertThat(assignment1.getLogicalSlotFuture().get() == assignment2.getLogicalSlotFuture().get(), is(true));
+	}
+
+	@Test
+	public void testFailedPhysicalSlotRequestFailsLogicalSlotFuturesAndRemovesSharedSlot() {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.completePhysicalSlotFutureManually()
+			.build();
+		CompletableFuture<LogicalSlot> logicalSlotFuture = context.allocateSlotsFor(0, 1).get(0).getLogicalSlotFuture();
+		SlotRequestId slotRequestId = context.getSlotProvider().getRequests().values().stream().findFirst().get().getSlotRequestId();
+
+		assertThat(logicalSlotFuture.isDone(), is(false));
+		context.getSlotProvider().failPhysicalSlotFutureFor(slotRequestId, new Throwable());
+		assertThat(logicalSlotFuture.isCompletedExceptionally(), is(true));
+
+		// next allocation allocates new shared slot
+		context.allocateSlotsFor(0, 1);
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyFalse() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(false);
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyTrue() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(true);
+	}
+
+	private static void testSlotWillBeOccupiedIndefinitely(boolean slotWillBeOccupiedIndefinitely) throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.setSlotWillBeOccupiedIndefinitely(slotWillBeOccupiedIndefinitely)
+			.build();
+		context.allocateSlotsFor(0, 1);
+
+		PhysicalSlotRequest slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst().get();
+		assertThat(slotRequest.willSlotBeOccupiedIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+
+		TestingPhysicalSlot physicalSlot = context.getSlotProvider().getResponses().get(slotRequest.getSlotRequestId()).get();
+		assertThat(physicalSlot.getPayload(), notNullValue());
+		assertThat(physicalSlot.getPayload().willOccupySlotIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+	}
+
+	@Test
+	public void testReturningLogicalSlotsRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			false,
+			(context, assignment) -> {
+				try {
+					assignment.getLogicalSlotFuture().get().releaseSlot(null);
+				} catch (InterruptedException | ExecutionException e) {
+					throw new FlinkRuntimeException("Unexpected", e);
+				}
+			});
+	}
+
+	@Test
+	public void testLogicalSlotCancelsPhysicalSlotRequestAndRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			true,
+			(context, assignment) -> {
+				context.getAllocator().cancel(assignment.getExecutionVertexId());
+				try {
+					assignment.getLogicalSlotFuture().get();
+					fail("THe logical future must finish with the cancellation exception");
+				} catch (InterruptedException | ExecutionException e) {
+					assertThat(e.getCause(), instanceOf(CancellationException.class));
+				}
+			});
+	}
+
+	private static void testLogicalSlotRequestCancellation(
+			boolean completePhysicalSlotFutureManually,
+			BiConsumer<AllocationContext, SlotExecutionVertexAssignment> cancelAction) {
+		//if (completePhysicalSlotRequest) {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(2)
+			.completePhysicalSlotFutureManually(completePhysicalSlotFutureManually)
+			.build();
+
+		List<SlotExecutionVertexAssignment> assignments = context.allocateSlotsFor(0, 2);
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+
+		// cancel only one sharing logical slots
+		cancelAction.accept(context, assignments.get(0));
+		assignments = context.allocateSlotsFor(0, 2);
+		// there should be no more physical slot allocations, as the first logical slot reuses the previous shared slot
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+
+		// cancel all sharing logical slots
+		for (SlotExecutionVertexAssignment assignment : assignments) {
+			cancelAction.accept(context, assignment);
+		}
+		SlotRequestId slotRequestId = context.getSlotProvider().getRequests().keySet().stream().findFirst().get();
+		assertThat(context.getSlotProvider().getCancelations().containsKey(slotRequestId), is(true));
+
+		context.allocateSlotsFor(0, 2);
+		// 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 testPhysicalSlotReleaseLogicalSlots() throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		List<SlotExecutionVertexAssignment> assignments = context.allocateSlotsFor(0, 2);
+		List<TestingPayload> payloads = assignments
+			.stream()
+			.map(assignment -> {
+				TestingPayload payload = new TestingPayload();
+				assignment.getLogicalSlotFuture().thenAccept(logicalSlot -> logicalSlot.tryAssignPayload(payload));
+				return payload;
+			})
+			.collect(Collectors.toList());
+		TestingPhysicalSlot physicalSlot = context.getSlotProvider().getResponses().values().stream().findFirst().get().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));
+	}
+
+	private static List<ExecutionVertexID> getAssignIds(Collection<SlotExecutionVertexAssignment> assignments) {
+		return assignments.stream().map(SlotExecutionVertexAssignment::getExecutionVertexId).collect(Collectors.toList());
+	}
+
+	private static class AllocationContext {
+		private final List<ExecutionVertexSchedulingRequirements> requirements;
+		private final TestingPhysicalSlotProvider slotProvider;
+		private final TestingSlotSharingStrategy slotSharingStrategy;
+		private final SlotSharingExecutionSlotAllocator allocator;
+		private final TestingSharedSlotProfileRetrieverFactory slotProfileRetrieverFactory;
+
+		AllocationContext(
+				List<ExecutionVertexSchedulingRequirements> requirements,
+				TestingPhysicalSlotProvider slotProvider,
+				TestingSlotSharingStrategy slotSharingStrategy,
+				SlotSharingExecutionSlotAllocator allocator,
+				TestingSharedSlotProfileRetrieverFactory slotProfileRetrieverFactory) {
+			this.requirements = requirements;
+			this.slotProvider = slotProvider;
+			this.slotSharingStrategy = slotSharingStrategy;
+			this.allocator = allocator;
+			this.slotProfileRetrieverFactory = slotProfileRetrieverFactory;
+		}
+
+		public SlotSharingExecutionSlotAllocator getAllocator() {
+			return allocator;
+		}
+
+		private List<SlotExecutionVertexAssignment> allocateSlotsFor(int start, int end) {
+			return allocator.allocateSlotsFor(requirements.subList(start, end));
+		}
+
+		private TestingSlotSharingStrategy getSlotSharingStrategy() {
+			return slotSharingStrategy;
+		}
+
+		private List<ExecutionVertexID> getReqIds(int start, int end) {
+			return getReqIds(requirements.subList(start, end));
+		}
+
+		private TestingPhysicalSlotProvider getSlotProvider() {
+			return slotProvider;
+		}
+
+		private TestingSharedSlotProfileRetrieverFactory getSlotProfileRetrieverFactory() {
+			return slotProfileRetrieverFactory;
+		}
+
+		private static List<ExecutionVertexID> getReqIds(Collection<ExecutionVertexSchedulingRequirements> requirements) {
+			return requirements.stream().map(ExecutionVertexSchedulingRequirements::getExecutionVertexId).collect(Collectors.toList());
+		}
+
+		static Builder newBuilder() {
+			return new Builder();
+		}
+
+		private static class Builder {
+			private int[] groups = { 2, 1 }; // 2 executions in the first group, 1 in the second etc
+			private List<ResourceProfile> resourceProfiles;
+			private boolean completePhysicalSlotFutureManually;
+			private boolean completeSlotProfileFutureManually;
+			private boolean slotWillBeOccupiedIndefinitely;
+
+			private Builder setGroups(int... groups) {
+				int reqNumber = IntStream.of(groups).sum();
+				List<ResourceProfile> resourceProfiles = Collections.nCopies(reqNumber, ResourceProfile.UNKNOWN);
+				return setGroupsWithResourceProfiles(resourceProfiles, groups);
+			}
+
+			private Builder setGroupsWithResourceProfiles(List<ResourceProfile> resourceProfiles, int... groups) {
+				Preconditions.checkArgument(resourceProfiles.size() == IntStream.of(groups).sum());
+				this.resourceProfiles = resourceProfiles;
+				this.groups = groups;
+				return this;
+			}
+
+			private Builder completePhysicalSlotFutureManually() {
+				completePhysicalSlotFutureManually(true);
+				return this;
+			}
+
+			private Builder completePhysicalSlotFutureManually(boolean value) {
+				this.completePhysicalSlotFutureManually = value;
+				return this;
+			}
+
+			private Builder completeSlotProfileFutureManually() {
+				this.completeSlotProfileFutureManually = true;
+				return this;
+			}
+
+			private Builder setSlotWillBeOccupiedIndefinitely(boolean slotWillBeOccupiedIndefinitely) {
+				this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+				return this;
+			}
+
+			private AllocationContext build() {
+				List<ExecutionVertexSchedulingRequirements> requirements = createSchedulingRequirements();
+				TestingPhysicalSlotProvider slotProvider = new TestingPhysicalSlotProvider(completePhysicalSlotFutureManually);
+				TestingSharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory =
+					new TestingSharedSlotProfileRetrieverFactory(completeSlotProfileFutureManually);
+				TestingSlotSharingStrategy slotSharingStrategy = TestingSlotSharingStrategy.createWithGroups(getReqIds(requirements), groups);
+				SlotSharingExecutionSlotAllocator allocator = new SlotSharingExecutionSlotAllocator(
+					slotProvider,
+					slotWillBeOccupiedIndefinitely,
+					slotSharingStrategy,
+					sharedSlotProfileRetrieverFactory);
+				return new AllocationContext(
+					requirements,
+					slotProvider,
+					slotSharingStrategy,
+					allocator,
+					sharedSlotProfileRetrieverFactory);
+			}
+
+			private List<ExecutionVertexSchedulingRequirements> createSchedulingRequirements() {
+				return resourceProfiles
+					.stream()
+					.map(resourceProfile ->
+						new ExecutionVertexSchedulingRequirements
+							.Builder()
+							.withExecutionVertexId(new ExecutionVertexID(new JobVertexID(), 0))
+							.withTaskResourceProfile(resourceProfile)
+							.withPhysicalSlotResourceProfile(resourceProfile) // not used
+							.build())
+					.collect(Collectors.toList());
+			}
+		}
+	}
+
+	private static class TestingPhysicalSlotProvider implements PhysicalSlotProvider {
+		private final Map<SlotRequestId, PhysicalSlotRequest> requests;
+		private final Map<SlotRequestId, CompletableFuture<TestingPhysicalSlot>> responses;
+		private final Map<SlotRequestId, Throwable> cancelations;

Review comment:
       typo: `cancelations` -> `cancellations`

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java
##########
@@ -0,0 +1,575 @@
+/*
+ * 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.clusterframework.types.SlotProfile;
+import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway;
+import org.apache.flink.runtime.instance.SimpleSlotContext;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.TestingPayload;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import org.junit.Test;
+import org.powermock.core.IdentityHashSet;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.BiConsumer;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Test suite for {@link SlotSharingExecutionSlotAllocator}.
+ */
+public class SlotSharingExecutionSlotAllocatorTest {
+	@Test
+	public void testSlotProfileRequestAskedBulkAndGroup() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		context.allocateSlotsFor(0, 2);
+
+		Set<ExecutionVertexID> ids = new HashSet<>(context.getReqIds(0, 2));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedBulks(), containsInAnyOrder(ids));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedGroups(), containsInAnyOrder(executionSlotSharingGroup));
+	}
+
+	@Test
+	public void testSlotRequestCompletionAfterProfileCompletion() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).completeSlotProfileFutureManually().build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 2);
+
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(false)));
+		context.getSlotProfileRetrieverFactory().completeSlotProfileFutureFor(executionSlotSharingGroup);
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(true)));
+	}
+
+	@Test
+	public void testSlotRequestProfile() {
+		ResourceProfile physicalsSlotResourceProfile = ResourceProfile.fromResources(3, 5);
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+		context.getSlotProfileRetrieverFactory().addGroupResourceProfile(executionSlotSharingGroup, physicalsSlotResourceProfile);
+
+		context.allocateSlotsFor(0, 2);
+
+		Optional<PhysicalSlotRequest> slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst();
+		assertThat(slotRequest.isPresent(), is(true));
+		slotRequest.ifPresent(r -> assertThat(r.getSlotProfile().getPhysicalSlotResourceProfile(), is(physicalsSlotResourceProfile)));
+	}
+
+	@Test
+	public void testNewAllocatePhysicalSlotForSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2, 2).build();
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 4);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(0, 4).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testAllocateLogicalSlotFromAvailableSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+
+		context.allocateSlotsFor(0, 1);
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(1, 2);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		// execution 0 from the first allocateSlotsFor call and execution 1 from the second allocateSlotsFor call
+		// share a slot, therefore only one physical slot allocation should happen
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(1, 2).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+	}
+
+	@Test
+	public void testDuplicateAllocationDoesNotRecreateLogicalSlotFuture() throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(1).build();
+
+		SlotExecutionVertexAssignment assignment1 = context.allocateSlotsFor(0, 1).get(0);
+		SlotExecutionVertexAssignment assignment2 = context.allocateSlotsFor(0, 1).get(0);
+
+		assertThat(assignment1.getLogicalSlotFuture().get() == assignment2.getLogicalSlotFuture().get(), is(true));
+	}
+
+	@Test
+	public void testFailedPhysicalSlotRequestFailsLogicalSlotFuturesAndRemovesSharedSlot() {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.completePhysicalSlotFutureManually()
+			.build();
+		CompletableFuture<LogicalSlot> logicalSlotFuture = context.allocateSlotsFor(0, 1).get(0).getLogicalSlotFuture();
+		SlotRequestId slotRequestId = context.getSlotProvider().getRequests().values().stream().findFirst().get().getSlotRequestId();
+
+		assertThat(logicalSlotFuture.isDone(), is(false));
+		context.getSlotProvider().failPhysicalSlotFutureFor(slotRequestId, new Throwable());
+		assertThat(logicalSlotFuture.isCompletedExceptionally(), is(true));
+
+		// next allocation allocates new shared slot
+		context.allocateSlotsFor(0, 1);
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyFalse() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(false);
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyTrue() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(true);
+	}
+
+	private static void testSlotWillBeOccupiedIndefinitely(boolean slotWillBeOccupiedIndefinitely) throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.setSlotWillBeOccupiedIndefinitely(slotWillBeOccupiedIndefinitely)
+			.build();
+		context.allocateSlotsFor(0, 1);
+
+		PhysicalSlotRequest slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst().get();
+		assertThat(slotRequest.willSlotBeOccupiedIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+
+		TestingPhysicalSlot physicalSlot = context.getSlotProvider().getResponses().get(slotRequest.getSlotRequestId()).get();
+		assertThat(physicalSlot.getPayload(), notNullValue());
+		assertThat(physicalSlot.getPayload().willOccupySlotIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+	}
+
+	@Test
+	public void testReturningLogicalSlotsRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			false,
+			(context, assignment) -> {
+				try {
+					assignment.getLogicalSlotFuture().get().releaseSlot(null);
+				} catch (InterruptedException | ExecutionException e) {
+					throw new FlinkRuntimeException("Unexpected", e);
+				}
+			});
+	}
+
+	@Test
+	public void testLogicalSlotCancelsPhysicalSlotRequestAndRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			true,
+			(context, assignment) -> {
+				context.getAllocator().cancel(assignment.getExecutionVertexId());
+				try {
+					assignment.getLogicalSlotFuture().get();
+					fail("THe logical future must finish with the cancellation exception");
+				} catch (InterruptedException | ExecutionException e) {
+					assertThat(e.getCause(), instanceOf(CancellationException.class));
+				}
+			});
+	}
+
+	private static void testLogicalSlotRequestCancellation(
+			boolean completePhysicalSlotFutureManually,
+			BiConsumer<AllocationContext, SlotExecutionVertexAssignment> cancelAction) {
+		//if (completePhysicalSlotRequest) {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(2)
+			.completePhysicalSlotFutureManually(completePhysicalSlotFutureManually)
+			.build();
+
+		List<SlotExecutionVertexAssignment> assignments = context.allocateSlotsFor(0, 2);
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+
+		// cancel only one sharing logical slots
+		cancelAction.accept(context, assignments.get(0));
+		assignments = context.allocateSlotsFor(0, 2);
+		// there should be no more physical slot allocations, as the first logical slot reuses the previous shared slot
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+
+		// cancel all sharing logical slots
+		for (SlotExecutionVertexAssignment assignment : assignments) {
+			cancelAction.accept(context, assignment);
+		}
+		SlotRequestId slotRequestId = context.getSlotProvider().getRequests().keySet().stream().findFirst().get();
+		assertThat(context.getSlotProvider().getCancelations().containsKey(slotRequestId), is(true));
+
+		context.allocateSlotsFor(0, 2);
+		// 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 testPhysicalSlotReleaseLogicalSlots() throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		List<SlotExecutionVertexAssignment> assignments = context.allocateSlotsFor(0, 2);
+		List<TestingPayload> payloads = assignments
+			.stream()
+			.map(assignment -> {
+				TestingPayload payload = new TestingPayload();
+				assignment.getLogicalSlotFuture().thenAccept(logicalSlot -> logicalSlot.tryAssignPayload(payload));
+				return payload;
+			})
+			.collect(Collectors.toList());
+		TestingPhysicalSlot physicalSlot = context.getSlotProvider().getResponses().values().stream().findFirst().get().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));
+	}
+
+	private static List<ExecutionVertexID> getAssignIds(Collection<SlotExecutionVertexAssignment> assignments) {
+		return assignments.stream().map(SlotExecutionVertexAssignment::getExecutionVertexId).collect(Collectors.toList());
+	}
+
+	private static class AllocationContext {
+		private final List<ExecutionVertexSchedulingRequirements> requirements;
+		private final TestingPhysicalSlotProvider slotProvider;
+		private final TestingSlotSharingStrategy slotSharingStrategy;
+		private final SlotSharingExecutionSlotAllocator allocator;
+		private final TestingSharedSlotProfileRetrieverFactory slotProfileRetrieverFactory;
+
+		AllocationContext(
+				List<ExecutionVertexSchedulingRequirements> requirements,
+				TestingPhysicalSlotProvider slotProvider,
+				TestingSlotSharingStrategy slotSharingStrategy,
+				SlotSharingExecutionSlotAllocator allocator,
+				TestingSharedSlotProfileRetrieverFactory slotProfileRetrieverFactory) {
+			this.requirements = requirements;
+			this.slotProvider = slotProvider;
+			this.slotSharingStrategy = slotSharingStrategy;
+			this.allocator = allocator;
+			this.slotProfileRetrieverFactory = slotProfileRetrieverFactory;
+		}
+
+		public SlotSharingExecutionSlotAllocator getAllocator() {
+			return allocator;
+		}
+
+		private List<SlotExecutionVertexAssignment> allocateSlotsFor(int start, int end) {
+			return allocator.allocateSlotsFor(requirements.subList(start, end));
+		}
+
+		private TestingSlotSharingStrategy getSlotSharingStrategy() {
+			return slotSharingStrategy;
+		}
+
+		private List<ExecutionVertexID> getReqIds(int start, int end) {
+			return getReqIds(requirements.subList(start, end));
+		}
+
+		private TestingPhysicalSlotProvider getSlotProvider() {
+			return slotProvider;
+		}
+
+		private TestingSharedSlotProfileRetrieverFactory getSlotProfileRetrieverFactory() {
+			return slotProfileRetrieverFactory;
+		}
+
+		private static List<ExecutionVertexID> getReqIds(Collection<ExecutionVertexSchedulingRequirements> requirements) {
+			return requirements.stream().map(ExecutionVertexSchedulingRequirements::getExecutionVertexId).collect(Collectors.toList());
+		}
+
+		static Builder newBuilder() {
+			return new Builder();
+		}
+
+		private static class Builder {
+			private int[] groups = { 2, 1 }; // 2 executions in the first group, 1 in the second etc
+			private List<ResourceProfile> resourceProfiles;
+			private boolean completePhysicalSlotFutureManually;
+			private boolean completeSlotProfileFutureManually;
+			private boolean slotWillBeOccupiedIndefinitely;
+
+			private Builder setGroups(int... groups) {
+				int reqNumber = IntStream.of(groups).sum();
+				List<ResourceProfile> resourceProfiles = Collections.nCopies(reqNumber, ResourceProfile.UNKNOWN);
+				return setGroupsWithResourceProfiles(resourceProfiles, groups);
+			}
+
+			private Builder setGroupsWithResourceProfiles(List<ResourceProfile> resourceProfiles, int... groups) {
+				Preconditions.checkArgument(resourceProfiles.size() == IntStream.of(groups).sum());
+				this.resourceProfiles = resourceProfiles;
+				this.groups = groups;
+				return this;
+			}
+
+			private Builder completePhysicalSlotFutureManually() {
+				completePhysicalSlotFutureManually(true);
+				return this;
+			}
+
+			private Builder completePhysicalSlotFutureManually(boolean value) {
+				this.completePhysicalSlotFutureManually = value;
+				return this;
+			}
+
+			private Builder completeSlotProfileFutureManually() {
+				this.completeSlotProfileFutureManually = true;
+				return this;
+			}
+
+			private Builder setSlotWillBeOccupiedIndefinitely(boolean slotWillBeOccupiedIndefinitely) {
+				this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+				return this;
+			}
+
+			private AllocationContext build() {
+				List<ExecutionVertexSchedulingRequirements> requirements = createSchedulingRequirements();
+				TestingPhysicalSlotProvider slotProvider = new TestingPhysicalSlotProvider(completePhysicalSlotFutureManually);
+				TestingSharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory =
+					new TestingSharedSlotProfileRetrieverFactory(completeSlotProfileFutureManually);
+				TestingSlotSharingStrategy slotSharingStrategy = TestingSlotSharingStrategy.createWithGroups(getReqIds(requirements), groups);
+				SlotSharingExecutionSlotAllocator allocator = new SlotSharingExecutionSlotAllocator(
+					slotProvider,
+					slotWillBeOccupiedIndefinitely,
+					slotSharingStrategy,
+					sharedSlotProfileRetrieverFactory);
+				return new AllocationContext(
+					requirements,
+					slotProvider,
+					slotSharingStrategy,
+					allocator,
+					sharedSlotProfileRetrieverFactory);
+			}
+
+			private List<ExecutionVertexSchedulingRequirements> createSchedulingRequirements() {
+				return resourceProfiles
+					.stream()
+					.map(resourceProfile ->
+						new ExecutionVertexSchedulingRequirements
+							.Builder()
+							.withExecutionVertexId(new ExecutionVertexID(new JobVertexID(), 0))
+							.withTaskResourceProfile(resourceProfile)
+							.withPhysicalSlotResourceProfile(resourceProfile) // not used
+							.build())
+					.collect(Collectors.toList());
+			}
+		}
+	}
+
+	private static class TestingPhysicalSlotProvider implements PhysicalSlotProvider {
+		private final Map<SlotRequestId, PhysicalSlotRequest> requests;
+		private final Map<SlotRequestId, CompletableFuture<TestingPhysicalSlot>> responses;
+		private final Map<SlotRequestId, Throwable> cancelations;
+		private final boolean completePhysicalSlotFutureManually;
+
+		private TestingPhysicalSlotProvider(boolean completePhysicalSlotFutureManually) {
+			this.completePhysicalSlotFutureManually = completePhysicalSlotFutureManually;
+			this.requests = new HashMap<>();
+			this.responses = new HashMap<>();
+			this.cancelations = new HashMap<>();
+		}
+
+		@Override
+		public CompletableFuture<PhysicalSlotRequest.Result> allocatePhysicalSlot(PhysicalSlotRequest physicalSlotRequest) {
+			SlotRequestId slotRequestId = physicalSlotRequest.getSlotRequestId();
+			requests.put(slotRequestId, physicalSlotRequest);
+			CompletableFuture<TestingPhysicalSlot> resultFuture = new CompletableFuture<>();
+			responses.put(slotRequestId, resultFuture);
+			if (!completePhysicalSlotFutureManually) {
+				completePhysicalSlotFutureFor(slotRequestId);
+			}
+			return resultFuture.thenApply(physicalSlot -> new PhysicalSlotRequest.Result(slotRequestId, physicalSlot));
+		}
+
+		private void completePhysicalSlotFutureFor(SlotRequestId slotRequestId) {
+			ResourceProfile resourceProfile = requests.get(slotRequestId).getSlotProfile().getPhysicalSlotResourceProfile();
+			TestingPhysicalSlot physicalSlot = new TestingPhysicalSlot(resourceProfile);
+			responses.get(slotRequestId).complete(physicalSlot);
+		}
+
+		private void failPhysicalSlotFutureFor(SlotRequestId slotRequestId, Throwable cause) {
+			responses.get(slotRequestId).completeExceptionally(cause);
+		}
+
+		@Override
+		public void cancelSlotRequest(SlotRequestId slotRequestId, Throwable cause) {
+			cancelations.put(slotRequestId, cause);
+		}
+
+		private Map<SlotRequestId, PhysicalSlotRequest> getRequests() {
+			return Collections.unmodifiableMap(requests);
+		}
+
+		private Map<SlotRequestId, CompletableFuture<TestingPhysicalSlot>> getResponses() {
+			return Collections.unmodifiableMap(responses);
+		}
+
+		private Map<SlotRequestId, Throwable> getCancelations() {
+			return Collections.unmodifiableMap(cancelations);
+		}
+	}
+
+	private static class TestingSlotSharingStrategy implements SlotSharingStrategy {
+		private final Map<ExecutionVertexID, ExecutionSlotSharingGroup> executionSlotSharingGroups;
+
+		private TestingSlotSharingStrategy(Map<ExecutionVertexID, ExecutionSlotSharingGroup> executionSlotSharingGroups) {
+			this.executionSlotSharingGroups = executionSlotSharingGroups;
+		}
+
+		@Override
+		public ExecutionSlotSharingGroup getExecutionSlotSharingGroup(ExecutionVertexID executionVertexId) {
+			return executionSlotSharingGroups.get(executionVertexId);
+		}
+
+		@Override
+		public Set<ExecutionSlotSharingGroup> getExecutionSlotSharingGroups() {
+			Set<ExecutionSlotSharingGroup> groups = new IdentityHashSet<>();
+			groups.addAll(executionSlotSharingGroups.values());
+			return groups;
+		}
+
+		private static TestingSlotSharingStrategy createWithGroups(
+				List<ExecutionVertexID> executionVertexIds,
+				int... groupSizes) {
+			Map<ExecutionVertexID, ExecutionSlotSharingGroup> executionSlotSharingGroups = new HashMap<>();
+			int startIndex = 0;
+			int nextIndex = 0;
+			for (int groupSize : groupSizes) {
+				nextIndex = startIndex + groupSize;
+				createGroup(executionSlotSharingGroups, executionVertexIds, startIndex, nextIndex);
+				startIndex = nextIndex;
+			}
+			if (nextIndex < executionVertexIds.size()) {
+				createGroup(executionSlotSharingGroups, executionVertexIds, nextIndex, executionVertexIds.size());
+			}
+			return new TestingSlotSharingStrategy(executionSlotSharingGroups);
+		}
+
+		private static void createGroup(
+				Map<ExecutionVertexID, ExecutionSlotSharingGroup> executionSlotSharingGroups,
+				List<ExecutionVertexID> executionVertexIds,
+				int startIndex,
+				int nextIndex) {
+			ExecutionSlotSharingGroup executionSlotSharingGroup = new ExecutionSlotSharingGroup();
+			executionSlotSharingGroup.addVertex(new ExecutionVertexID(new JobVertexID(), 0));
+			executionVertexIds.subList(startIndex, nextIndex).forEach(executionVertexId -> {
+				executionSlotSharingGroup.addVertex(executionVertexId);
+				executionSlotSharingGroups.put(executionVertexId, executionSlotSharingGroup);
+			});
+		}
+	}
+
+	private static class TestingSharedSlotProfileRetrieverFactory implements SharedSlotProfileRetrieverFactory {
+		private final List<Set<ExecutionVertexID>> askedBulks;
+		private final List<ExecutionSlotSharingGroup> askedGroups;
+		private final Map<ExecutionSlotSharingGroup, ResourceProfile> resourceProfiles;
+		private final Map<ExecutionSlotSharingGroup, CompletableFuture<SlotProfile>> slotProfileFutures;
+		private final boolean completeSlotProfileFutureManually;
+
+		private TestingSharedSlotProfileRetrieverFactory(boolean completeSlotProfileFutureManually) {
+			this.completeSlotProfileFutureManually = completeSlotProfileFutureManually;
+			this.askedBulks = new ArrayList<>();
+			this.askedGroups = new ArrayList<>();
+			this.resourceProfiles = new IdentityHashMap<>();
+			this.slotProfileFutures = new IdentityHashMap<>();
+		}
+
+		@Override
+		public SharedSlotProfileRetriever createFromBulk(Set<ExecutionVertexID> bulk) {
+			askedBulks.add(bulk);
+			return group -> {
+				askedGroups.add(group);
+				CompletableFuture<SlotProfile> slotProfileFuture =
+					slotProfileFutures.computeIfAbsent(group, g -> new CompletableFuture<>());
+				if (!completeSlotProfileFutureManually) {
+					completeSlotProfileFutureFor(group);
+				}
+				return slotProfileFuture;
+			};
+		}
+
+		private void addGroupResourceProfile(ExecutionSlotSharingGroup group, ResourceProfile resourceProfile) {
+			resourceProfiles.put(group, resourceProfile);
+		}
+
+		private void completeSlotProfileFutureFor(ExecutionSlotSharingGroup group) {
+			slotProfileFutures.get(group).complete(SlotProfile.noLocality(resourceProfiles.getOrDefault(group, ResourceProfile.ANY)));
+		}
+
+		private List<Set<ExecutionVertexID>> getAskedBulks() {
+			return Collections.unmodifiableList(askedBulks);
+		}
+
+		private List<ExecutionSlotSharingGroup> getAskedGroups() {
+			return Collections.unmodifiableList(askedGroups);
+		}
+	}
+
+	private static class TestingPhysicalSlot extends SimpleSlotContext implements PhysicalSlot {
+		private Payload payload;

Review comment:
       should be marked as `@Nullable`

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java
##########
@@ -0,0 +1,575 @@
+/*
+ * 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.clusterframework.types.SlotProfile;
+import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway;
+import org.apache.flink.runtime.instance.SimpleSlotContext;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.TestingPayload;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import org.junit.Test;
+import org.powermock.core.IdentityHashSet;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.BiConsumer;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Test suite for {@link SlotSharingExecutionSlotAllocator}.
+ */
+public class SlotSharingExecutionSlotAllocatorTest {
+	@Test
+	public void testSlotProfileRequestAskedBulkAndGroup() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		context.allocateSlotsFor(0, 2);
+
+		Set<ExecutionVertexID> ids = new HashSet<>(context.getReqIds(0, 2));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedBulks(), containsInAnyOrder(ids));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedGroups(), containsInAnyOrder(executionSlotSharingGroup));
+	}
+
+	@Test
+	public void testSlotRequestCompletionAfterProfileCompletion() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).completeSlotProfileFutureManually().build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 2);
+
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(false)));
+		context.getSlotProfileRetrieverFactory().completeSlotProfileFutureFor(executionSlotSharingGroup);
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(true)));
+	}
+
+	@Test
+	public void testSlotRequestProfile() {
+		ResourceProfile physicalsSlotResourceProfile = ResourceProfile.fromResources(3, 5);
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+		context.getSlotProfileRetrieverFactory().addGroupResourceProfile(executionSlotSharingGroup, physicalsSlotResourceProfile);
+
+		context.allocateSlotsFor(0, 2);
+
+		Optional<PhysicalSlotRequest> slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst();
+		assertThat(slotRequest.isPresent(), is(true));
+		slotRequest.ifPresent(r -> assertThat(r.getSlotProfile().getPhysicalSlotResourceProfile(), is(physicalsSlotResourceProfile)));
+	}
+
+	@Test
+	public void testNewAllocatePhysicalSlotForSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2, 2).build();
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 4);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(0, 4).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testAllocateLogicalSlotFromAvailableSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+
+		context.allocateSlotsFor(0, 1);
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(1, 2);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		// execution 0 from the first allocateSlotsFor call and execution 1 from the second allocateSlotsFor call
+		// share a slot, therefore only one physical slot allocation should happen
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(1, 2).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+	}
+
+	@Test
+	public void testDuplicateAllocationDoesNotRecreateLogicalSlotFuture() throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(1).build();
+
+		SlotExecutionVertexAssignment assignment1 = context.allocateSlotsFor(0, 1).get(0);
+		SlotExecutionVertexAssignment assignment2 = context.allocateSlotsFor(0, 1).get(0);
+
+		assertThat(assignment1.getLogicalSlotFuture().get() == assignment2.getLogicalSlotFuture().get(), is(true));
+	}
+
+	@Test
+	public void testFailedPhysicalSlotRequestFailsLogicalSlotFuturesAndRemovesSharedSlot() {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.completePhysicalSlotFutureManually()
+			.build();
+		CompletableFuture<LogicalSlot> logicalSlotFuture = context.allocateSlotsFor(0, 1).get(0).getLogicalSlotFuture();
+		SlotRequestId slotRequestId = context.getSlotProvider().getRequests().values().stream().findFirst().get().getSlotRequestId();
+
+		assertThat(logicalSlotFuture.isDone(), is(false));
+		context.getSlotProvider().failPhysicalSlotFutureFor(slotRequestId, new Throwable());
+		assertThat(logicalSlotFuture.isCompletedExceptionally(), is(true));
+
+		// next allocation allocates new shared slot
+		context.allocateSlotsFor(0, 1);
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyFalse() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(false);
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyTrue() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(true);
+	}
+
+	private static void testSlotWillBeOccupiedIndefinitely(boolean slotWillBeOccupiedIndefinitely) throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.setSlotWillBeOccupiedIndefinitely(slotWillBeOccupiedIndefinitely)
+			.build();
+		context.allocateSlotsFor(0, 1);
+
+		PhysicalSlotRequest slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst().get();
+		assertThat(slotRequest.willSlotBeOccupiedIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+
+		TestingPhysicalSlot physicalSlot = context.getSlotProvider().getResponses().get(slotRequest.getSlotRequestId()).get();
+		assertThat(physicalSlot.getPayload(), notNullValue());
+		assertThat(physicalSlot.getPayload().willOccupySlotIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+	}
+
+	@Test
+	public void testReturningLogicalSlotsRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			false,
+			(context, assignment) -> {
+				try {
+					assignment.getLogicalSlotFuture().get().releaseSlot(null);
+				} catch (InterruptedException | ExecutionException e) {
+					throw new FlinkRuntimeException("Unexpected", e);
+				}
+			});
+	}
+
+	@Test
+	public void testLogicalSlotCancelsPhysicalSlotRequestAndRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			true,
+			(context, assignment) -> {
+				context.getAllocator().cancel(assignment.getExecutionVertexId());
+				try {
+					assignment.getLogicalSlotFuture().get();
+					fail("THe logical future must finish with the cancellation exception");
+				} catch (InterruptedException | ExecutionException e) {
+					assertThat(e.getCause(), instanceOf(CancellationException.class));
+				}
+			});
+	}
+
+	private static void testLogicalSlotRequestCancellation(
+			boolean completePhysicalSlotFutureManually,
+			BiConsumer<AllocationContext, SlotExecutionVertexAssignment> cancelAction) {
+		//if (completePhysicalSlotRequest) {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(2)
+			.completePhysicalSlotFutureManually(completePhysicalSlotFutureManually)
+			.build();
+
+		List<SlotExecutionVertexAssignment> assignments = context.allocateSlotsFor(0, 2);
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+
+		// cancel only one sharing logical slots
+		cancelAction.accept(context, assignments.get(0));
+		assignments = context.allocateSlotsFor(0, 2);
+		// there should be no more physical slot allocations, as the first logical slot reuses the previous shared slot
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+
+		// cancel all sharing logical slots
+		for (SlotExecutionVertexAssignment assignment : assignments) {
+			cancelAction.accept(context, assignment);
+		}
+		SlotRequestId slotRequestId = context.getSlotProvider().getRequests().keySet().stream().findFirst().get();
+		assertThat(context.getSlotProvider().getCancelations().containsKey(slotRequestId), is(true));
+
+		context.allocateSlotsFor(0, 2);
+		// 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 testPhysicalSlotReleaseLogicalSlots() throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		List<SlotExecutionVertexAssignment> assignments = context.allocateSlotsFor(0, 2);
+		List<TestingPayload> payloads = assignments
+			.stream()
+			.map(assignment -> {
+				TestingPayload payload = new TestingPayload();
+				assignment.getLogicalSlotFuture().thenAccept(logicalSlot -> logicalSlot.tryAssignPayload(payload));
+				return payload;
+			})
+			.collect(Collectors.toList());
+		TestingPhysicalSlot physicalSlot = context.getSlotProvider().getResponses().values().stream().findFirst().get().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));
+	}
+
+	private static List<ExecutionVertexID> getAssignIds(Collection<SlotExecutionVertexAssignment> assignments) {
+		return assignments.stream().map(SlotExecutionVertexAssignment::getExecutionVertexId).collect(Collectors.toList());
+	}
+
+	private static class AllocationContext {
+		private final List<ExecutionVertexSchedulingRequirements> requirements;
+		private final TestingPhysicalSlotProvider slotProvider;
+		private final TestingSlotSharingStrategy slotSharingStrategy;
+		private final SlotSharingExecutionSlotAllocator allocator;
+		private final TestingSharedSlotProfileRetrieverFactory slotProfileRetrieverFactory;
+
+		AllocationContext(
+				List<ExecutionVertexSchedulingRequirements> requirements,
+				TestingPhysicalSlotProvider slotProvider,
+				TestingSlotSharingStrategy slotSharingStrategy,
+				SlotSharingExecutionSlotAllocator allocator,
+				TestingSharedSlotProfileRetrieverFactory slotProfileRetrieverFactory) {
+			this.requirements = requirements;
+			this.slotProvider = slotProvider;
+			this.slotSharingStrategy = slotSharingStrategy;
+			this.allocator = allocator;
+			this.slotProfileRetrieverFactory = slotProfileRetrieverFactory;
+		}
+
+		public SlotSharingExecutionSlotAllocator getAllocator() {
+			return allocator;
+		}
+
+		private List<SlotExecutionVertexAssignment> allocateSlotsFor(int start, int end) {
+			return allocator.allocateSlotsFor(requirements.subList(start, end));
+		}
+
+		private TestingSlotSharingStrategy getSlotSharingStrategy() {
+			return slotSharingStrategy;
+		}
+
+		private List<ExecutionVertexID> getReqIds(int start, int end) {
+			return getReqIds(requirements.subList(start, end));
+		}
+
+		private TestingPhysicalSlotProvider getSlotProvider() {
+			return slotProvider;
+		}
+
+		private TestingSharedSlotProfileRetrieverFactory getSlotProfileRetrieverFactory() {
+			return slotProfileRetrieverFactory;
+		}
+
+		private static List<ExecutionVertexID> getReqIds(Collection<ExecutionVertexSchedulingRequirements> requirements) {
+			return requirements.stream().map(ExecutionVertexSchedulingRequirements::getExecutionVertexId).collect(Collectors.toList());
+		}
+
+		static Builder newBuilder() {
+			return new Builder();
+		}
+
+		private static class Builder {
+			private int[] groups = { 2, 1 }; // 2 executions in the first group, 1 in the second etc
+			private List<ResourceProfile> resourceProfiles;
+			private boolean completePhysicalSlotFutureManually;
+			private boolean completeSlotProfileFutureManually;
+			private boolean slotWillBeOccupiedIndefinitely;
+
+			private Builder setGroups(int... groups) {
+				int reqNumber = IntStream.of(groups).sum();
+				List<ResourceProfile> resourceProfiles = Collections.nCopies(reqNumber, ResourceProfile.UNKNOWN);
+				return setGroupsWithResourceProfiles(resourceProfiles, groups);
+			}
+
+			private Builder setGroupsWithResourceProfiles(List<ResourceProfile> resourceProfiles, int... groups) {
+				Preconditions.checkArgument(resourceProfiles.size() == IntStream.of(groups).sum());
+				this.resourceProfiles = resourceProfiles;
+				this.groups = groups;
+				return this;
+			}
+
+			private Builder completePhysicalSlotFutureManually() {
+				completePhysicalSlotFutureManually(true);
+				return this;
+			}
+
+			private Builder completePhysicalSlotFutureManually(boolean value) {
+				this.completePhysicalSlotFutureManually = value;
+				return this;
+			}
+
+			private Builder completeSlotProfileFutureManually() {
+				this.completeSlotProfileFutureManually = true;
+				return this;
+			}
+
+			private Builder setSlotWillBeOccupiedIndefinitely(boolean slotWillBeOccupiedIndefinitely) {
+				this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+				return this;
+			}
+
+			private AllocationContext build() {
+				List<ExecutionVertexSchedulingRequirements> requirements = createSchedulingRequirements();
+				TestingPhysicalSlotProvider slotProvider = new TestingPhysicalSlotProvider(completePhysicalSlotFutureManually);
+				TestingSharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory =
+					new TestingSharedSlotProfileRetrieverFactory(completeSlotProfileFutureManually);
+				TestingSlotSharingStrategy slotSharingStrategy = TestingSlotSharingStrategy.createWithGroups(getReqIds(requirements), groups);
+				SlotSharingExecutionSlotAllocator allocator = new SlotSharingExecutionSlotAllocator(
+					slotProvider,
+					slotWillBeOccupiedIndefinitely,
+					slotSharingStrategy,
+					sharedSlotProfileRetrieverFactory);
+				return new AllocationContext(
+					requirements,
+					slotProvider,
+					slotSharingStrategy,
+					allocator,
+					sharedSlotProfileRetrieverFactory);
+			}
+
+			private List<ExecutionVertexSchedulingRequirements> createSchedulingRequirements() {
+				return resourceProfiles
+					.stream()
+					.map(resourceProfile ->
+						new ExecutionVertexSchedulingRequirements
+							.Builder()
+							.withExecutionVertexId(new ExecutionVertexID(new JobVertexID(), 0))
+							.withTaskResourceProfile(resourceProfile)
+							.withPhysicalSlotResourceProfile(resourceProfile) // not used
+							.build())
+					.collect(Collectors.toList());
+			}
+		}
+	}
+
+	private static class TestingPhysicalSlotProvider implements PhysicalSlotProvider {
+		private final Map<SlotRequestId, PhysicalSlotRequest> requests;
+		private final Map<SlotRequestId, CompletableFuture<TestingPhysicalSlot>> responses;
+		private final Map<SlotRequestId, Throwable> cancelations;
+		private final boolean completePhysicalSlotFutureManually;
+
+		private TestingPhysicalSlotProvider(boolean completePhysicalSlotFutureManually) {
+			this.completePhysicalSlotFutureManually = completePhysicalSlotFutureManually;
+			this.requests = new HashMap<>();
+			this.responses = new HashMap<>();
+			this.cancelations = new HashMap<>();
+		}
+
+		@Override
+		public CompletableFuture<PhysicalSlotRequest.Result> allocatePhysicalSlot(PhysicalSlotRequest physicalSlotRequest) {
+			SlotRequestId slotRequestId = physicalSlotRequest.getSlotRequestId();
+			requests.put(slotRequestId, physicalSlotRequest);
+			CompletableFuture<TestingPhysicalSlot> resultFuture = new CompletableFuture<>();
+			responses.put(slotRequestId, resultFuture);
+			if (!completePhysicalSlotFutureManually) {
+				completePhysicalSlotFutureFor(slotRequestId);
+			}
+			return resultFuture.thenApply(physicalSlot -> new PhysicalSlotRequest.Result(slotRequestId, physicalSlot));
+		}
+
+		private void completePhysicalSlotFutureFor(SlotRequestId slotRequestId) {
+			ResourceProfile resourceProfile = requests.get(slotRequestId).getSlotProfile().getPhysicalSlotResourceProfile();
+			TestingPhysicalSlot physicalSlot = new TestingPhysicalSlot(resourceProfile);
+			responses.get(slotRequestId).complete(physicalSlot);
+		}
+
+		private void failPhysicalSlotFutureFor(SlotRequestId slotRequestId, Throwable cause) {
+			responses.get(slotRequestId).completeExceptionally(cause);
+		}
+
+		@Override
+		public void cancelSlotRequest(SlotRequestId slotRequestId, Throwable cause) {
+			cancelations.put(slotRequestId, cause);
+		}
+
+		private Map<SlotRequestId, PhysicalSlotRequest> getRequests() {
+			return Collections.unmodifiableMap(requests);
+		}
+
+		private Map<SlotRequestId, CompletableFuture<TestingPhysicalSlot>> getResponses() {
+			return Collections.unmodifiableMap(responses);
+		}
+
+		private Map<SlotRequestId, Throwable> getCancelations() {
+			return Collections.unmodifiableMap(cancelations);
+		}
+	}
+
+	private static class TestingSlotSharingStrategy implements SlotSharingStrategy {
+		private final Map<ExecutionVertexID, ExecutionSlotSharingGroup> executionSlotSharingGroups;
+
+		private TestingSlotSharingStrategy(Map<ExecutionVertexID, ExecutionSlotSharingGroup> executionSlotSharingGroups) {
+			this.executionSlotSharingGroups = executionSlotSharingGroups;
+		}
+
+		@Override
+		public ExecutionSlotSharingGroup getExecutionSlotSharingGroup(ExecutionVertexID executionVertexId) {
+			return executionSlotSharingGroups.get(executionVertexId);
+		}
+
+		@Override
+		public Set<ExecutionSlotSharingGroup> getExecutionSlotSharingGroups() {
+			Set<ExecutionSlotSharingGroup> groups = new IdentityHashSet<>();
+			groups.addAll(executionSlotSharingGroups.values());
+			return groups;
+		}
+
+		private static TestingSlotSharingStrategy createWithGroups(
+				List<ExecutionVertexID> executionVertexIds,
+				int... groupSizes) {
+			Map<ExecutionVertexID, ExecutionSlotSharingGroup> executionSlotSharingGroups = new HashMap<>();
+			int startIndex = 0;
+			int nextIndex = 0;
+			for (int groupSize : groupSizes) {
+				nextIndex = startIndex + groupSize;
+				createGroup(executionSlotSharingGroups, executionVertexIds, startIndex, nextIndex);
+				startIndex = nextIndex;
+			}
+			if (nextIndex < executionVertexIds.size()) {
+				createGroup(executionSlotSharingGroups, executionVertexIds, nextIndex, executionVertexIds.size());
+			}
+			return new TestingSlotSharingStrategy(executionSlotSharingGroups);
+		}
+
+		private static void createGroup(
+				Map<ExecutionVertexID, ExecutionSlotSharingGroup> executionSlotSharingGroups,
+				List<ExecutionVertexID> executionVertexIds,
+				int startIndex,
+				int nextIndex) {
+			ExecutionSlotSharingGroup executionSlotSharingGroup = new ExecutionSlotSharingGroup();
+			executionSlotSharingGroup.addVertex(new ExecutionVertexID(new JobVertexID(), 0));

Review comment:
       I guess this statement is added by mistake?
   All tests can pass without it.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java
##########
@@ -0,0 +1,575 @@
+/*
+ * 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.clusterframework.types.SlotProfile;
+import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway;
+import org.apache.flink.runtime.instance.SimpleSlotContext;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.TestingPayload;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import org.junit.Test;
+import org.powermock.core.IdentityHashSet;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.BiConsumer;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Test suite for {@link SlotSharingExecutionSlotAllocator}.
+ */
+public class SlotSharingExecutionSlotAllocatorTest {
+	@Test
+	public void testSlotProfileRequestAskedBulkAndGroup() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		context.allocateSlotsFor(0, 2);
+
+		Set<ExecutionVertexID> ids = new HashSet<>(context.getReqIds(0, 2));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedBulks(), containsInAnyOrder(ids));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedGroups(), containsInAnyOrder(executionSlotSharingGroup));
+	}
+
+	@Test
+	public void testSlotRequestCompletionAfterProfileCompletion() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).completeSlotProfileFutureManually().build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 2);
+
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(false)));
+		context.getSlotProfileRetrieverFactory().completeSlotProfileFutureFor(executionSlotSharingGroup);
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(true)));
+	}
+
+	@Test
+	public void testSlotRequestProfile() {
+		ResourceProfile physicalsSlotResourceProfile = ResourceProfile.fromResources(3, 5);
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+		context.getSlotProfileRetrieverFactory().addGroupResourceProfile(executionSlotSharingGroup, physicalsSlotResourceProfile);
+
+		context.allocateSlotsFor(0, 2);
+
+		Optional<PhysicalSlotRequest> slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst();
+		assertThat(slotRequest.isPresent(), is(true));
+		slotRequest.ifPresent(r -> assertThat(r.getSlotProfile().getPhysicalSlotResourceProfile(), is(physicalsSlotResourceProfile)));
+	}
+
+	@Test
+	public void testNewAllocatePhysicalSlotForSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2, 2).build();
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 4);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(0, 4).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testAllocateLogicalSlotFromAvailableSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+
+		context.allocateSlotsFor(0, 1);
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(1, 2);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		// execution 0 from the first allocateSlotsFor call and execution 1 from the second allocateSlotsFor call
+		// share a slot, therefore only one physical slot allocation should happen
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(1, 2).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+	}
+
+	@Test
+	public void testDuplicateAllocationDoesNotRecreateLogicalSlotFuture() throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(1).build();
+
+		SlotExecutionVertexAssignment assignment1 = context.allocateSlotsFor(0, 1).get(0);
+		SlotExecutionVertexAssignment assignment2 = context.allocateSlotsFor(0, 1).get(0);
+
+		assertThat(assignment1.getLogicalSlotFuture().get() == assignment2.getLogicalSlotFuture().get(), is(true));
+	}
+
+	@Test
+	public void testFailedPhysicalSlotRequestFailsLogicalSlotFuturesAndRemovesSharedSlot() {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.completePhysicalSlotFutureManually()
+			.build();
+		CompletableFuture<LogicalSlot> logicalSlotFuture = context.allocateSlotsFor(0, 1).get(0).getLogicalSlotFuture();
+		SlotRequestId slotRequestId = context.getSlotProvider().getRequests().values().stream().findFirst().get().getSlotRequestId();
+
+		assertThat(logicalSlotFuture.isDone(), is(false));
+		context.getSlotProvider().failPhysicalSlotFutureFor(slotRequestId, new Throwable());
+		assertThat(logicalSlotFuture.isCompletedExceptionally(), is(true));
+
+		// next allocation allocates new shared slot
+		context.allocateSlotsFor(0, 1);
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyFalse() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(false);
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyTrue() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(true);
+	}
+
+	private static void testSlotWillBeOccupiedIndefinitely(boolean slotWillBeOccupiedIndefinitely) throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.setSlotWillBeOccupiedIndefinitely(slotWillBeOccupiedIndefinitely)
+			.build();
+		context.allocateSlotsFor(0, 1);
+
+		PhysicalSlotRequest slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst().get();
+		assertThat(slotRequest.willSlotBeOccupiedIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+
+		TestingPhysicalSlot physicalSlot = context.getSlotProvider().getResponses().get(slotRequest.getSlotRequestId()).get();
+		assertThat(physicalSlot.getPayload(), notNullValue());
+		assertThat(physicalSlot.getPayload().willOccupySlotIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+	}
+
+	@Test
+	public void testReturningLogicalSlotsRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			false,
+			(context, assignment) -> {
+				try {
+					assignment.getLogicalSlotFuture().get().releaseSlot(null);
+				} catch (InterruptedException | ExecutionException e) {
+					throw new FlinkRuntimeException("Unexpected", e);
+				}
+			});
+	}
+
+	@Test
+	public void testLogicalSlotCancelsPhysicalSlotRequestAndRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			true,
+			(context, assignment) -> {
+				context.getAllocator().cancel(assignment.getExecutionVertexId());
+				try {
+					assignment.getLogicalSlotFuture().get();
+					fail("THe logical future must finish with the cancellation exception");
+				} catch (InterruptedException | ExecutionException e) {
+					assertThat(e.getCause(), instanceOf(CancellationException.class));
+				}
+			});
+	}
+
+	private static void testLogicalSlotRequestCancellation(
+			boolean completePhysicalSlotFutureManually,
+			BiConsumer<AllocationContext, SlotExecutionVertexAssignment> cancelAction) {
+		//if (completePhysicalSlotRequest) {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(2)
+			.completePhysicalSlotFutureManually(completePhysicalSlotFutureManually)
+			.build();
+
+		List<SlotExecutionVertexAssignment> assignments = context.allocateSlotsFor(0, 2);
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+
+		// cancel only one sharing logical slots
+		cancelAction.accept(context, assignments.get(0));
+		assignments = context.allocateSlotsFor(0, 2);
+		// there should be no more physical slot allocations, as the first logical slot reuses the previous shared slot
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+
+		// cancel all sharing logical slots
+		for (SlotExecutionVertexAssignment assignment : assignments) {
+			cancelAction.accept(context, assignment);
+		}
+		SlotRequestId slotRequestId = context.getSlotProvider().getRequests().keySet().stream().findFirst().get();
+		assertThat(context.getSlotProvider().getCancelations().containsKey(slotRequestId), is(true));
+
+		context.allocateSlotsFor(0, 2);
+		// 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 testPhysicalSlotReleaseLogicalSlots() throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		List<SlotExecutionVertexAssignment> assignments = context.allocateSlotsFor(0, 2);
+		List<TestingPayload> payloads = assignments
+			.stream()
+			.map(assignment -> {
+				TestingPayload payload = new TestingPayload();
+				assignment.getLogicalSlotFuture().thenAccept(logicalSlot -> logicalSlot.tryAssignPayload(payload));
+				return payload;
+			})
+			.collect(Collectors.toList());
+		TestingPhysicalSlot physicalSlot = context.getSlotProvider().getResponses().values().stream().findFirst().get().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));
+	}
+
+	private static List<ExecutionVertexID> getAssignIds(Collection<SlotExecutionVertexAssignment> assignments) {
+		return assignments.stream().map(SlotExecutionVertexAssignment::getExecutionVertexId).collect(Collectors.toList());
+	}
+
+	private static class AllocationContext {
+		private final List<ExecutionVertexSchedulingRequirements> requirements;
+		private final TestingPhysicalSlotProvider slotProvider;
+		private final TestingSlotSharingStrategy slotSharingStrategy;
+		private final SlotSharingExecutionSlotAllocator allocator;
+		private final TestingSharedSlotProfileRetrieverFactory slotProfileRetrieverFactory;
+
+		AllocationContext(
+				List<ExecutionVertexSchedulingRequirements> requirements,
+				TestingPhysicalSlotProvider slotProvider,
+				TestingSlotSharingStrategy slotSharingStrategy,
+				SlotSharingExecutionSlotAllocator allocator,
+				TestingSharedSlotProfileRetrieverFactory slotProfileRetrieverFactory) {
+			this.requirements = requirements;
+			this.slotProvider = slotProvider;
+			this.slotSharingStrategy = slotSharingStrategy;
+			this.allocator = allocator;
+			this.slotProfileRetrieverFactory = slotProfileRetrieverFactory;
+		}
+
+		public SlotSharingExecutionSlotAllocator getAllocator() {

Review comment:
       can be `private`

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java
##########
@@ -0,0 +1,575 @@
+/*
+ * 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.clusterframework.types.SlotProfile;
+import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway;
+import org.apache.flink.runtime.instance.SimpleSlotContext;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.TestingPayload;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import org.junit.Test;
+import org.powermock.core.IdentityHashSet;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.BiConsumer;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Test suite for {@link SlotSharingExecutionSlotAllocator}.
+ */
+public class SlotSharingExecutionSlotAllocatorTest {
+	@Test
+	public void testSlotProfileRequestAskedBulkAndGroup() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		context.allocateSlotsFor(0, 2);
+
+		Set<ExecutionVertexID> ids = new HashSet<>(context.getReqIds(0, 2));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedBulks(), containsInAnyOrder(ids));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedGroups(), containsInAnyOrder(executionSlotSharingGroup));
+	}
+
+	@Test
+	public void testSlotRequestCompletionAfterProfileCompletion() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).completeSlotProfileFutureManually().build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 2);
+
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(false)));
+		context.getSlotProfileRetrieverFactory().completeSlotProfileFutureFor(executionSlotSharingGroup);
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(true)));
+	}
+
+	@Test
+	public void testSlotRequestProfile() {
+		ResourceProfile physicalsSlotResourceProfile = ResourceProfile.fromResources(3, 5);
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+		context.getSlotProfileRetrieverFactory().addGroupResourceProfile(executionSlotSharingGroup, physicalsSlotResourceProfile);
+
+		context.allocateSlotsFor(0, 2);
+
+		Optional<PhysicalSlotRequest> slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst();
+		assertThat(slotRequest.isPresent(), is(true));
+		slotRequest.ifPresent(r -> assertThat(r.getSlotProfile().getPhysicalSlotResourceProfile(), is(physicalsSlotResourceProfile)));
+	}
+
+	@Test
+	public void testNewAllocatePhysicalSlotForSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2, 2).build();
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 4);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(0, 4).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testAllocateLogicalSlotFromAvailableSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+
+		context.allocateSlotsFor(0, 1);
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(1, 2);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		// execution 0 from the first allocateSlotsFor call and execution 1 from the second allocateSlotsFor call
+		// share a slot, therefore only one physical slot allocation should happen
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(1, 2).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+	}
+
+	@Test
+	public void testDuplicateAllocationDoesNotRecreateLogicalSlotFuture() throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(1).build();
+
+		SlotExecutionVertexAssignment assignment1 = context.allocateSlotsFor(0, 1).get(0);
+		SlotExecutionVertexAssignment assignment2 = context.allocateSlotsFor(0, 1).get(0);
+
+		assertThat(assignment1.getLogicalSlotFuture().get() == assignment2.getLogicalSlotFuture().get(), is(true));
+	}
+
+	@Test
+	public void testFailedPhysicalSlotRequestFailsLogicalSlotFuturesAndRemovesSharedSlot() {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.completePhysicalSlotFutureManually()
+			.build();
+		CompletableFuture<LogicalSlot> logicalSlotFuture = context.allocateSlotsFor(0, 1).get(0).getLogicalSlotFuture();
+		SlotRequestId slotRequestId = context.getSlotProvider().getRequests().values().stream().findFirst().get().getSlotRequestId();
+
+		assertThat(logicalSlotFuture.isDone(), is(false));
+		context.getSlotProvider().failPhysicalSlotFutureFor(slotRequestId, new Throwable());
+		assertThat(logicalSlotFuture.isCompletedExceptionally(), is(true));
+
+		// next allocation allocates new shared slot
+		context.allocateSlotsFor(0, 1);
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyFalse() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(false);
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyTrue() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(true);
+	}
+
+	private static void testSlotWillBeOccupiedIndefinitely(boolean slotWillBeOccupiedIndefinitely) throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.setSlotWillBeOccupiedIndefinitely(slotWillBeOccupiedIndefinitely)
+			.build();
+		context.allocateSlotsFor(0, 1);
+
+		PhysicalSlotRequest slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst().get();
+		assertThat(slotRequest.willSlotBeOccupiedIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+
+		TestingPhysicalSlot physicalSlot = context.getSlotProvider().getResponses().get(slotRequest.getSlotRequestId()).get();
+		assertThat(physicalSlot.getPayload(), notNullValue());
+		assertThat(physicalSlot.getPayload().willOccupySlotIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+	}
+
+	@Test
+	public void testReturningLogicalSlotsRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			false,
+			(context, assignment) -> {
+				try {
+					assignment.getLogicalSlotFuture().get().releaseSlot(null);
+				} catch (InterruptedException | ExecutionException e) {
+					throw new FlinkRuntimeException("Unexpected", e);
+				}
+			});
+	}
+
+	@Test
+	public void testLogicalSlotCancelsPhysicalSlotRequestAndRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			true,
+			(context, assignment) -> {
+				context.getAllocator().cancel(assignment.getExecutionVertexId());
+				try {
+					assignment.getLogicalSlotFuture().get();
+					fail("THe logical future must finish with the cancellation exception");
+				} catch (InterruptedException | ExecutionException e) {
+					assertThat(e.getCause(), instanceOf(CancellationException.class));
+				}
+			});
+	}
+
+	private static void testLogicalSlotRequestCancellation(
+			boolean completePhysicalSlotFutureManually,
+			BiConsumer<AllocationContext, SlotExecutionVertexAssignment> cancelAction) {
+		//if (completePhysicalSlotRequest) {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(2)
+			.completePhysicalSlotFutureManually(completePhysicalSlotFutureManually)
+			.build();
+
+		List<SlotExecutionVertexAssignment> assignments = context.allocateSlotsFor(0, 2);
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+
+		// cancel only one sharing logical slots
+		cancelAction.accept(context, assignments.get(0));
+		assignments = context.allocateSlotsFor(0, 2);
+		// there should be no more physical slot allocations, as the first logical slot reuses the previous shared slot
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+
+		// cancel all sharing logical slots
+		for (SlotExecutionVertexAssignment assignment : assignments) {
+			cancelAction.accept(context, assignment);
+		}
+		SlotRequestId slotRequestId = context.getSlotProvider().getRequests().keySet().stream().findFirst().get();
+		assertThat(context.getSlotProvider().getCancelations().containsKey(slotRequestId), is(true));
+
+		context.allocateSlotsFor(0, 2);
+		// 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 testPhysicalSlotReleaseLogicalSlots() throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		List<SlotExecutionVertexAssignment> assignments = context.allocateSlotsFor(0, 2);
+		List<TestingPayload> payloads = assignments
+			.stream()
+			.map(assignment -> {
+				TestingPayload payload = new TestingPayload();
+				assignment.getLogicalSlotFuture().thenAccept(logicalSlot -> logicalSlot.tryAssignPayload(payload));
+				return payload;
+			})
+			.collect(Collectors.toList());
+		TestingPhysicalSlot physicalSlot = context.getSlotProvider().getResponses().values().stream().findFirst().get().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));
+	}
+
+	private static List<ExecutionVertexID> getAssignIds(Collection<SlotExecutionVertexAssignment> assignments) {
+		return assignments.stream().map(SlotExecutionVertexAssignment::getExecutionVertexId).collect(Collectors.toList());
+	}
+
+	private static class AllocationContext {
+		private final List<ExecutionVertexSchedulingRequirements> requirements;
+		private final TestingPhysicalSlotProvider slotProvider;
+		private final TestingSlotSharingStrategy slotSharingStrategy;
+		private final SlotSharingExecutionSlotAllocator allocator;
+		private final TestingSharedSlotProfileRetrieverFactory slotProfileRetrieverFactory;
+
+		AllocationContext(
+				List<ExecutionVertexSchedulingRequirements> requirements,
+				TestingPhysicalSlotProvider slotProvider,
+				TestingSlotSharingStrategy slotSharingStrategy,
+				SlotSharingExecutionSlotAllocator allocator,
+				TestingSharedSlotProfileRetrieverFactory slotProfileRetrieverFactory) {
+			this.requirements = requirements;
+			this.slotProvider = slotProvider;
+			this.slotSharingStrategy = slotSharingStrategy;
+			this.allocator = allocator;
+			this.slotProfileRetrieverFactory = slotProfileRetrieverFactory;
+		}
+
+		public SlotSharingExecutionSlotAllocator getAllocator() {
+			return allocator;
+		}
+
+		private List<SlotExecutionVertexAssignment> allocateSlotsFor(int start, int end) {
+			return allocator.allocateSlotsFor(requirements.subList(start, end));
+		}
+
+		private TestingSlotSharingStrategy getSlotSharingStrategy() {
+			return slotSharingStrategy;
+		}
+
+		private List<ExecutionVertexID> getReqIds(int start, int end) {
+			return getReqIds(requirements.subList(start, end));
+		}
+
+		private TestingPhysicalSlotProvider getSlotProvider() {
+			return slotProvider;
+		}
+
+		private TestingSharedSlotProfileRetrieverFactory getSlotProfileRetrieverFactory() {
+			return slotProfileRetrieverFactory;
+		}
+
+		private static List<ExecutionVertexID> getReqIds(Collection<ExecutionVertexSchedulingRequirements> requirements) {
+			return requirements.stream().map(ExecutionVertexSchedulingRequirements::getExecutionVertexId).collect(Collectors.toList());
+		}
+
+		static Builder newBuilder() {
+			return new Builder();
+		}
+
+		private static class Builder {
+			private int[] groups = { 2, 1 }; // 2 executions in the first group, 1 in the second etc
+			private List<ResourceProfile> resourceProfiles;
+			private boolean completePhysicalSlotFutureManually;
+			private boolean completeSlotProfileFutureManually;
+			private boolean slotWillBeOccupiedIndefinitely;
+
+			private Builder setGroups(int... groups) {
+				int reqNumber = IntStream.of(groups).sum();
+				List<ResourceProfile> resourceProfiles = Collections.nCopies(reqNumber, ResourceProfile.UNKNOWN);
+				return setGroupsWithResourceProfiles(resourceProfiles, groups);
+			}
+
+			private Builder setGroupsWithResourceProfiles(List<ResourceProfile> resourceProfiles, int... groups) {
+				Preconditions.checkArgument(resourceProfiles.size() == IntStream.of(groups).sum());
+				this.resourceProfiles = resourceProfiles;
+				this.groups = groups;
+				return this;
+			}
+
+			private Builder completePhysicalSlotFutureManually() {
+				completePhysicalSlotFutureManually(true);
+				return this;
+			}
+
+			private Builder completePhysicalSlotFutureManually(boolean value) {
+				this.completePhysicalSlotFutureManually = value;
+				return this;
+			}
+
+			private Builder completeSlotProfileFutureManually() {
+				this.completeSlotProfileFutureManually = true;
+				return this;
+			}
+
+			private Builder setSlotWillBeOccupiedIndefinitely(boolean slotWillBeOccupiedIndefinitely) {
+				this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+				return this;
+			}
+
+			private AllocationContext build() {
+				List<ExecutionVertexSchedulingRequirements> requirements = createSchedulingRequirements();
+				TestingPhysicalSlotProvider slotProvider = new TestingPhysicalSlotProvider(completePhysicalSlotFutureManually);
+				TestingSharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory =
+					new TestingSharedSlotProfileRetrieverFactory(completeSlotProfileFutureManually);
+				TestingSlotSharingStrategy slotSharingStrategy = TestingSlotSharingStrategy.createWithGroups(getReqIds(requirements), groups);
+				SlotSharingExecutionSlotAllocator allocator = new SlotSharingExecutionSlotAllocator(
+					slotProvider,
+					slotWillBeOccupiedIndefinitely,
+					slotSharingStrategy,
+					sharedSlotProfileRetrieverFactory);
+				return new AllocationContext(
+					requirements,
+					slotProvider,
+					slotSharingStrategy,
+					allocator,
+					sharedSlotProfileRetrieverFactory);
+			}
+
+			private List<ExecutionVertexSchedulingRequirements> createSchedulingRequirements() {
+				return resourceProfiles
+					.stream()
+					.map(resourceProfile ->
+						new ExecutionVertexSchedulingRequirements
+							.Builder()
+							.withExecutionVertexId(new ExecutionVertexID(new JobVertexID(), 0))
+							.withTaskResourceProfile(resourceProfile)

Review comment:
       Both `taskResourceProfile` and `physicalSlotResourceProfile` of `ExecutionVertexSchedulingRequirements` are not used in `SlotSharingExecutionSlotAllocator`. I think we do not need to set them and therefore no need to create the `resourceProfiles`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java
##########
@@ -0,0 +1,575 @@
+/*
+ * 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.clusterframework.types.SlotProfile;
+import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway;
+import org.apache.flink.runtime.instance.SimpleSlotContext;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.TestingPayload;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import org.junit.Test;
+import org.powermock.core.IdentityHashSet;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.BiConsumer;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Test suite for {@link SlotSharingExecutionSlotAllocator}.
+ */
+public class SlotSharingExecutionSlotAllocatorTest {
+	@Test
+	public void testSlotProfileRequestAskedBulkAndGroup() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		context.allocateSlotsFor(0, 2);
+
+		Set<ExecutionVertexID> ids = new HashSet<>(context.getReqIds(0, 2));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedBulks(), containsInAnyOrder(ids));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedGroups(), containsInAnyOrder(executionSlotSharingGroup));
+	}
+
+	@Test
+	public void testSlotRequestCompletionAfterProfileCompletion() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).completeSlotProfileFutureManually().build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 2);
+
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(false)));
+		context.getSlotProfileRetrieverFactory().completeSlotProfileFutureFor(executionSlotSharingGroup);
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(true)));
+	}
+
+	@Test
+	public void testSlotRequestProfile() {
+		ResourceProfile physicalsSlotResourceProfile = ResourceProfile.fromResources(3, 5);
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+		context.getSlotProfileRetrieverFactory().addGroupResourceProfile(executionSlotSharingGroup, physicalsSlotResourceProfile);
+
+		context.allocateSlotsFor(0, 2);
+
+		Optional<PhysicalSlotRequest> slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst();
+		assertThat(slotRequest.isPresent(), is(true));
+		slotRequest.ifPresent(r -> assertThat(r.getSlotProfile().getPhysicalSlotResourceProfile(), is(physicalsSlotResourceProfile)));
+	}
+
+	@Test
+	public void testNewAllocatePhysicalSlotForSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2, 2).build();
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 4);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(0, 4).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testAllocateLogicalSlotFromAvailableSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+
+		context.allocateSlotsFor(0, 1);
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(1, 2);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		// execution 0 from the first allocateSlotsFor call and execution 1 from the second allocateSlotsFor call
+		// share a slot, therefore only one physical slot allocation should happen
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(1, 2).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+	}
+
+	@Test
+	public void testDuplicateAllocationDoesNotRecreateLogicalSlotFuture() throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(1).build();
+
+		SlotExecutionVertexAssignment assignment1 = context.allocateSlotsFor(0, 1).get(0);
+		SlotExecutionVertexAssignment assignment2 = context.allocateSlotsFor(0, 1).get(0);
+
+		assertThat(assignment1.getLogicalSlotFuture().get() == assignment2.getLogicalSlotFuture().get(), is(true));
+	}
+
+	@Test
+	public void testFailedPhysicalSlotRequestFailsLogicalSlotFuturesAndRemovesSharedSlot() {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.completePhysicalSlotFutureManually()
+			.build();
+		CompletableFuture<LogicalSlot> logicalSlotFuture = context.allocateSlotsFor(0, 1).get(0).getLogicalSlotFuture();
+		SlotRequestId slotRequestId = context.getSlotProvider().getRequests().values().stream().findFirst().get().getSlotRequestId();
+
+		assertThat(logicalSlotFuture.isDone(), is(false));
+		context.getSlotProvider().failPhysicalSlotFutureFor(slotRequestId, new Throwable());
+		assertThat(logicalSlotFuture.isCompletedExceptionally(), is(true));
+
+		// next allocation allocates new shared slot
+		context.allocateSlotsFor(0, 1);
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyFalse() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(false);
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyTrue() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(true);
+	}
+
+	private static void testSlotWillBeOccupiedIndefinitely(boolean slotWillBeOccupiedIndefinitely) throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.setSlotWillBeOccupiedIndefinitely(slotWillBeOccupiedIndefinitely)
+			.build();
+		context.allocateSlotsFor(0, 1);
+
+		PhysicalSlotRequest slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst().get();
+		assertThat(slotRequest.willSlotBeOccupiedIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+
+		TestingPhysicalSlot physicalSlot = context.getSlotProvider().getResponses().get(slotRequest.getSlotRequestId()).get();
+		assertThat(physicalSlot.getPayload(), notNullValue());
+		assertThat(physicalSlot.getPayload().willOccupySlotIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+	}
+
+	@Test
+	public void testReturningLogicalSlotsRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			false,
+			(context, assignment) -> {
+				try {
+					assignment.getLogicalSlotFuture().get().releaseSlot(null);
+				} catch (InterruptedException | ExecutionException e) {
+					throw new FlinkRuntimeException("Unexpected", e);
+				}
+			});
+	}
+
+	@Test
+	public void testLogicalSlotCancelsPhysicalSlotRequestAndRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			true,
+			(context, assignment) -> {
+				context.getAllocator().cancel(assignment.getExecutionVertexId());
+				try {
+					assignment.getLogicalSlotFuture().get();
+					fail("THe logical future must finish with the cancellation exception");
+				} catch (InterruptedException | ExecutionException e) {
+					assertThat(e.getCause(), instanceOf(CancellationException.class));
+				}
+			});
+	}
+
+	private static void testLogicalSlotRequestCancellation(
+			boolean completePhysicalSlotFutureManually,
+			BiConsumer<AllocationContext, SlotExecutionVertexAssignment> cancelAction) {
+		//if (completePhysicalSlotRequest) {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(2)
+			.completePhysicalSlotFutureManually(completePhysicalSlotFutureManually)
+			.build();
+
+		List<SlotExecutionVertexAssignment> assignments = context.allocateSlotsFor(0, 2);
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+
+		// cancel only one sharing logical slots
+		cancelAction.accept(context, assignments.get(0));
+		assignments = context.allocateSlotsFor(0, 2);
+		// there should be no more physical slot allocations, as the first logical slot reuses the previous shared slot
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+
+		// cancel all sharing logical slots
+		for (SlotExecutionVertexAssignment assignment : assignments) {
+			cancelAction.accept(context, assignment);
+		}
+		SlotRequestId slotRequestId = context.getSlotProvider().getRequests().keySet().stream().findFirst().get();
+		assertThat(context.getSlotProvider().getCancelations().containsKey(slotRequestId), is(true));
+
+		context.allocateSlotsFor(0, 2);
+		// 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 testPhysicalSlotReleaseLogicalSlots() throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		List<SlotExecutionVertexAssignment> assignments = context.allocateSlotsFor(0, 2);
+		List<TestingPayload> payloads = assignments
+			.stream()
+			.map(assignment -> {
+				TestingPayload payload = new TestingPayload();
+				assignment.getLogicalSlotFuture().thenAccept(logicalSlot -> logicalSlot.tryAssignPayload(payload));
+				return payload;
+			})
+			.collect(Collectors.toList());
+		TestingPhysicalSlot physicalSlot = context.getSlotProvider().getResponses().values().stream().findFirst().get().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));
+	}
+
+	private static List<ExecutionVertexID> getAssignIds(Collection<SlotExecutionVertexAssignment> assignments) {
+		return assignments.stream().map(SlotExecutionVertexAssignment::getExecutionVertexId).collect(Collectors.toList());
+	}
+
+	private static class AllocationContext {
+		private final List<ExecutionVertexSchedulingRequirements> requirements;
+		private final TestingPhysicalSlotProvider slotProvider;
+		private final TestingSlotSharingStrategy slotSharingStrategy;
+		private final SlotSharingExecutionSlotAllocator allocator;
+		private final TestingSharedSlotProfileRetrieverFactory slotProfileRetrieverFactory;
+
+		AllocationContext(

Review comment:
       can be `private`

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java
##########
@@ -0,0 +1,575 @@
+/*
+ * 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.clusterframework.types.SlotProfile;
+import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway;
+import org.apache.flink.runtime.instance.SimpleSlotContext;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.TestingPayload;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import org.junit.Test;
+import org.powermock.core.IdentityHashSet;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.BiConsumer;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Test suite for {@link SlotSharingExecutionSlotAllocator}.
+ */
+public class SlotSharingExecutionSlotAllocatorTest {
+	@Test
+	public void testSlotProfileRequestAskedBulkAndGroup() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();

Review comment:
       I feel that using indices here and there makes it a bit hard for reading and reasoning.
   How about let the test case decide the grouping in a more direct way.
   
   Below is an example showing a way that I feel more readable. I also implies some implementation changes on `AllocationContext`.
   
   ```
   public class SlotSharingExecutionSlotAllocatorTest {
   
   	private static final ExecutionVertexID EV1 = createRandomExecutionVertexID();
   	private static final ExecutionVertexID EV2 = createRandomExecutionVertexID();
   	private static final ExecutionVertexID EV3 = createRandomExecutionVertexID();
   	private static final ExecutionVertexID EV4 = createRandomExecutionVertexID();
   
   	@Test
   	public void testNewAllocatePhysicalSlotForSharedSlot() {
   		List<List<ExecutionVertexID>> vertices = Arrays.asList(
   			Arrays.asList(EV1, EV2),
   			Arrays.asList(EV3, EV4));
   		AllocationContext context = AllocationContext.newBuilder().setVertices(vertices).build();
   
   		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(EV1, EV2, EV3, EV4);
   		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
   
   		assertThat(assignIds, containsInAnyOrder(EV1, EV2, EV3, EV4));
   		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
   	}
   
   	@Test
   	public void testSlotProfileRequestAskedBulkAndGroup() {
   		List<ExecutionVertexID> vertexGroup1 = Arrays.asList(EV1, EV2);
   		List<List<ExecutionVertexID>> vertices = Arrays.asList(vertexGroup1);
   		AllocationContext context = AllocationContext.newBuilder().setVertices(vertices).build();
   		ExecutionSlotSharingGroup executionSlotSharingGroup =
   			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(EV1);
   
   		context.allocateSlotsFor(EV1, EV2);
   
   		Set<ExecutionVertexID> ids = new HashSet<>(vertexGroup1);
   		assertThat(context.getSlotProfileRetrieverFactory().getAskedBulks(), containsInAnyOrder(ids));
   		assertThat(context.getSlotProfileRetrieverFactory().getAskedGroups(), containsInAnyOrder(executionSlotSharingGroup));
   	}
   ...
   ```

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java
##########
@@ -0,0 +1,575 @@
+/*
+ * 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.clusterframework.types.SlotProfile;
+import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway;
+import org.apache.flink.runtime.instance.SimpleSlotContext;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.TestingPayload;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import org.junit.Test;
+import org.powermock.core.IdentityHashSet;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.BiConsumer;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Test suite for {@link SlotSharingExecutionSlotAllocator}.
+ */
+public class SlotSharingExecutionSlotAllocatorTest {
+	@Test
+	public void testSlotProfileRequestAskedBulkAndGroup() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		context.allocateSlotsFor(0, 2);
+
+		Set<ExecutionVertexID> ids = new HashSet<>(context.getReqIds(0, 2));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedBulks(), containsInAnyOrder(ids));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedGroups(), containsInAnyOrder(executionSlotSharingGroup));
+	}
+
+	@Test
+	public void testSlotRequestCompletionAfterProfileCompletion() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).completeSlotProfileFutureManually().build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 2);
+
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(false)));
+		context.getSlotProfileRetrieverFactory().completeSlotProfileFutureFor(executionSlotSharingGroup);
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(true)));
+	}
+
+	@Test
+	public void testSlotRequestProfile() {
+		ResourceProfile physicalsSlotResourceProfile = ResourceProfile.fromResources(3, 5);
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+		context.getSlotProfileRetrieverFactory().addGroupResourceProfile(executionSlotSharingGroup, physicalsSlotResourceProfile);
+
+		context.allocateSlotsFor(0, 2);
+
+		Optional<PhysicalSlotRequest> slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst();
+		assertThat(slotRequest.isPresent(), is(true));
+		slotRequest.ifPresent(r -> assertThat(r.getSlotProfile().getPhysicalSlotResourceProfile(), is(physicalsSlotResourceProfile)));
+	}
+
+	@Test
+	public void testNewAllocatePhysicalSlotForSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2, 2).build();
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 4);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(0, 4).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testAllocateLogicalSlotFromAvailableSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+
+		context.allocateSlotsFor(0, 1);
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(1, 2);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		// execution 0 from the first allocateSlotsFor call and execution 1 from the second allocateSlotsFor call
+		// share a slot, therefore only one physical slot allocation should happen
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(1, 2).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+	}
+
+	@Test
+	public void testDuplicateAllocationDoesNotRecreateLogicalSlotFuture() throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(1).build();
+
+		SlotExecutionVertexAssignment assignment1 = context.allocateSlotsFor(0, 1).get(0);
+		SlotExecutionVertexAssignment assignment2 = context.allocateSlotsFor(0, 1).get(0);
+
+		assertThat(assignment1.getLogicalSlotFuture().get() == assignment2.getLogicalSlotFuture().get(), is(true));
+	}
+
+	@Test
+	public void testFailedPhysicalSlotRequestFailsLogicalSlotFuturesAndRemovesSharedSlot() {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.completePhysicalSlotFutureManually()
+			.build();
+		CompletableFuture<LogicalSlot> logicalSlotFuture = context.allocateSlotsFor(0, 1).get(0).getLogicalSlotFuture();
+		SlotRequestId slotRequestId = context.getSlotProvider().getRequests().values().stream().findFirst().get().getSlotRequestId();
+
+		assertThat(logicalSlotFuture.isDone(), is(false));
+		context.getSlotProvider().failPhysicalSlotFutureFor(slotRequestId, new Throwable());
+		assertThat(logicalSlotFuture.isCompletedExceptionally(), is(true));
+
+		// next allocation allocates new shared slot
+		context.allocateSlotsFor(0, 1);
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyFalse() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(false);
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyTrue() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(true);
+	}
+
+	private static void testSlotWillBeOccupiedIndefinitely(boolean slotWillBeOccupiedIndefinitely) throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.setSlotWillBeOccupiedIndefinitely(slotWillBeOccupiedIndefinitely)
+			.build();
+		context.allocateSlotsFor(0, 1);
+
+		PhysicalSlotRequest slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst().get();
+		assertThat(slotRequest.willSlotBeOccupiedIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+
+		TestingPhysicalSlot physicalSlot = context.getSlotProvider().getResponses().get(slotRequest.getSlotRequestId()).get();
+		assertThat(physicalSlot.getPayload(), notNullValue());
+		assertThat(physicalSlot.getPayload().willOccupySlotIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+	}
+
+	@Test
+	public void testReturningLogicalSlotsRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			false,
+			(context, assignment) -> {
+				try {
+					assignment.getLogicalSlotFuture().get().releaseSlot(null);
+				} catch (InterruptedException | ExecutionException e) {
+					throw new FlinkRuntimeException("Unexpected", e);
+				}
+			});
+	}
+
+	@Test
+	public void testLogicalSlotCancelsPhysicalSlotRequestAndRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			true,
+			(context, assignment) -> {
+				context.getAllocator().cancel(assignment.getExecutionVertexId());
+				try {
+					assignment.getLogicalSlotFuture().get();
+					fail("THe logical future must finish with the cancellation exception");
+				} catch (InterruptedException | ExecutionException e) {
+					assertThat(e.getCause(), instanceOf(CancellationException.class));
+				}
+			});
+	}
+
+	private static void testLogicalSlotRequestCancellation(
+			boolean completePhysicalSlotFutureManually,
+			BiConsumer<AllocationContext, SlotExecutionVertexAssignment> cancelAction) {
+		//if (completePhysicalSlotRequest) {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(2)
+			.completePhysicalSlotFutureManually(completePhysicalSlotFutureManually)
+			.build();
+
+		List<SlotExecutionVertexAssignment> assignments = context.allocateSlotsFor(0, 2);
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+
+		// cancel only one sharing logical slots
+		cancelAction.accept(context, assignments.get(0));
+		assignments = context.allocateSlotsFor(0, 2);
+		// there should be no more physical slot allocations, as the first logical slot reuses the previous shared slot
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+
+		// cancel all sharing logical slots
+		for (SlotExecutionVertexAssignment assignment : assignments) {
+			cancelAction.accept(context, assignment);
+		}
+		SlotRequestId slotRequestId = context.getSlotProvider().getRequests().keySet().stream().findFirst().get();
+		assertThat(context.getSlotProvider().getCancelations().containsKey(slotRequestId), is(true));
+
+		context.allocateSlotsFor(0, 2);
+		// 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 testPhysicalSlotReleaseLogicalSlots() throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		List<SlotExecutionVertexAssignment> assignments = context.allocateSlotsFor(0, 2);
+		List<TestingPayload> payloads = assignments
+			.stream()
+			.map(assignment -> {
+				TestingPayload payload = new TestingPayload();
+				assignment.getLogicalSlotFuture().thenAccept(logicalSlot -> logicalSlot.tryAssignPayload(payload));
+				return payload;
+			})
+			.collect(Collectors.toList());
+		TestingPhysicalSlot physicalSlot = context.getSlotProvider().getResponses().values().stream().findFirst().get().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));
+	}
+
+	private static List<ExecutionVertexID> getAssignIds(Collection<SlotExecutionVertexAssignment> assignments) {
+		return assignments.stream().map(SlotExecutionVertexAssignment::getExecutionVertexId).collect(Collectors.toList());
+	}
+
+	private static class AllocationContext {
+		private final List<ExecutionVertexSchedulingRequirements> requirements;
+		private final TestingPhysicalSlotProvider slotProvider;
+		private final TestingSlotSharingStrategy slotSharingStrategy;
+		private final SlotSharingExecutionSlotAllocator allocator;
+		private final TestingSharedSlotProfileRetrieverFactory slotProfileRetrieverFactory;
+
+		AllocationContext(
+				List<ExecutionVertexSchedulingRequirements> requirements,
+				TestingPhysicalSlotProvider slotProvider,
+				TestingSlotSharingStrategy slotSharingStrategy,
+				SlotSharingExecutionSlotAllocator allocator,
+				TestingSharedSlotProfileRetrieverFactory slotProfileRetrieverFactory) {
+			this.requirements = requirements;
+			this.slotProvider = slotProvider;
+			this.slotSharingStrategy = slotSharingStrategy;
+			this.allocator = allocator;
+			this.slotProfileRetrieverFactory = slotProfileRetrieverFactory;
+		}
+
+		public SlotSharingExecutionSlotAllocator getAllocator() {
+			return allocator;
+		}
+
+		private List<SlotExecutionVertexAssignment> allocateSlotsFor(int start, int end) {
+			return allocator.allocateSlotsFor(requirements.subList(start, end));
+		}
+
+		private TestingSlotSharingStrategy getSlotSharingStrategy() {
+			return slotSharingStrategy;
+		}
+
+		private List<ExecutionVertexID> getReqIds(int start, int end) {
+			return getReqIds(requirements.subList(start, end));
+		}
+
+		private TestingPhysicalSlotProvider getSlotProvider() {
+			return slotProvider;
+		}
+
+		private TestingSharedSlotProfileRetrieverFactory getSlotProfileRetrieverFactory() {
+			return slotProfileRetrieverFactory;
+		}
+
+		private static List<ExecutionVertexID> getReqIds(Collection<ExecutionVertexSchedulingRequirements> requirements) {
+			return requirements.stream().map(ExecutionVertexSchedulingRequirements::getExecutionVertexId).collect(Collectors.toList());
+		}
+
+		static Builder newBuilder() {

Review comment:
       can be `private`

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java
##########
@@ -0,0 +1,575 @@
+/*
+ * 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.clusterframework.types.SlotProfile;
+import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway;
+import org.apache.flink.runtime.instance.SimpleSlotContext;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.TestingPayload;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import org.junit.Test;
+import org.powermock.core.IdentityHashSet;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.BiConsumer;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Test suite for {@link SlotSharingExecutionSlotAllocator}.
+ */
+public class SlotSharingExecutionSlotAllocatorTest {
+	@Test
+	public void testSlotProfileRequestAskedBulkAndGroup() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		context.allocateSlotsFor(0, 2);
+
+		Set<ExecutionVertexID> ids = new HashSet<>(context.getReqIds(0, 2));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedBulks(), containsInAnyOrder(ids));
+		assertThat(context.getSlotProfileRetrieverFactory().getAskedGroups(), containsInAnyOrder(executionSlotSharingGroup));
+	}
+
+	@Test
+	public void testSlotRequestCompletionAfterProfileCompletion() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).completeSlotProfileFutureManually().build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 2);
+
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(false)));
+		context.getSlotProfileRetrieverFactory().completeSlotProfileFutureFor(executionSlotSharingGroup);
+		executionVertexAssignments.forEach(assignment -> assertThat(assignment.getLogicalSlotFuture().isDone(), is(true)));
+	}
+
+	@Test
+	public void testSlotRequestProfile() {
+		ResourceProfile physicalsSlotResourceProfile = ResourceProfile.fromResources(3, 5);
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			context.getSlotSharingStrategy().getExecutionSlotSharingGroup(context.getReqIds(0, 1).get(0));
+		context.getSlotProfileRetrieverFactory().addGroupResourceProfile(executionSlotSharingGroup, physicalsSlotResourceProfile);
+
+		context.allocateSlotsFor(0, 2);
+
+		Optional<PhysicalSlotRequest> slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst();
+		assertThat(slotRequest.isPresent(), is(true));
+		slotRequest.ifPresent(r -> assertThat(r.getSlotProfile().getPhysicalSlotResourceProfile(), is(physicalsSlotResourceProfile)));
+	}
+
+	@Test
+	public void testNewAllocatePhysicalSlotForSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2, 2).build();
+
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(0, 4);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(0, 4).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testAllocateLogicalSlotFromAvailableSharedSlot() {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+
+		context.allocateSlotsFor(0, 1);
+		List<SlotExecutionVertexAssignment> executionVertexAssignments = context.allocateSlotsFor(1, 2);
+		Collection<ExecutionVertexID> assignIds = getAssignIds(executionVertexAssignments);
+
+		// execution 0 from the first allocateSlotsFor call and execution 1 from the second allocateSlotsFor call
+		// share a slot, therefore only one physical slot allocation should happen
+		assertThat(assignIds, containsInAnyOrder(context.getReqIds(1, 2).toArray()));
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+	}
+
+	@Test
+	public void testDuplicateAllocationDoesNotRecreateLogicalSlotFuture() throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(1).build();
+
+		SlotExecutionVertexAssignment assignment1 = context.allocateSlotsFor(0, 1).get(0);
+		SlotExecutionVertexAssignment assignment2 = context.allocateSlotsFor(0, 1).get(0);
+
+		assertThat(assignment1.getLogicalSlotFuture().get() == assignment2.getLogicalSlotFuture().get(), is(true));
+	}
+
+	@Test
+	public void testFailedPhysicalSlotRequestFailsLogicalSlotFuturesAndRemovesSharedSlot() {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.completePhysicalSlotFutureManually()
+			.build();
+		CompletableFuture<LogicalSlot> logicalSlotFuture = context.allocateSlotsFor(0, 1).get(0).getLogicalSlotFuture();
+		SlotRequestId slotRequestId = context.getSlotProvider().getRequests().values().stream().findFirst().get().getSlotRequestId();
+
+		assertThat(logicalSlotFuture.isDone(), is(false));
+		context.getSlotProvider().failPhysicalSlotFutureFor(slotRequestId, new Throwable());
+		assertThat(logicalSlotFuture.isCompletedExceptionally(), is(true));
+
+		// next allocation allocates new shared slot
+		context.allocateSlotsFor(0, 1);
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(2));
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyFalse() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(false);
+	}
+
+	@Test
+	public void testSlotWillBeOccupiedIndefinitelyTrue() throws ExecutionException, InterruptedException {
+		testSlotWillBeOccupiedIndefinitely(true);
+	}
+
+	private static void testSlotWillBeOccupiedIndefinitely(boolean slotWillBeOccupiedIndefinitely) throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(1)
+			.setSlotWillBeOccupiedIndefinitely(slotWillBeOccupiedIndefinitely)
+			.build();
+		context.allocateSlotsFor(0, 1);
+
+		PhysicalSlotRequest slotRequest = context.getSlotProvider().getRequests().values().stream().findFirst().get();
+		assertThat(slotRequest.willSlotBeOccupiedIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+
+		TestingPhysicalSlot physicalSlot = context.getSlotProvider().getResponses().get(slotRequest.getSlotRequestId()).get();
+		assertThat(physicalSlot.getPayload(), notNullValue());
+		assertThat(physicalSlot.getPayload().willOccupySlotIndefinitely(), is(slotWillBeOccupiedIndefinitely));
+	}
+
+	@Test
+	public void testReturningLogicalSlotsRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			false,
+			(context, assignment) -> {
+				try {
+					assignment.getLogicalSlotFuture().get().releaseSlot(null);
+				} catch (InterruptedException | ExecutionException e) {
+					throw new FlinkRuntimeException("Unexpected", e);
+				}
+			});
+	}
+
+	@Test
+	public void testLogicalSlotCancelsPhysicalSlotRequestAndRemovesSharedSlot() {
+		testLogicalSlotRequestCancellation(
+			true,
+			(context, assignment) -> {
+				context.getAllocator().cancel(assignment.getExecutionVertexId());
+				try {
+					assignment.getLogicalSlotFuture().get();
+					fail("THe logical future must finish with the cancellation exception");
+				} catch (InterruptedException | ExecutionException e) {
+					assertThat(e.getCause(), instanceOf(CancellationException.class));
+				}
+			});
+	}
+
+	private static void testLogicalSlotRequestCancellation(
+			boolean completePhysicalSlotFutureManually,
+			BiConsumer<AllocationContext, SlotExecutionVertexAssignment> cancelAction) {
+		//if (completePhysicalSlotRequest) {
+		AllocationContext context = AllocationContext
+			.newBuilder()
+			.setGroups(2)
+			.completePhysicalSlotFutureManually(completePhysicalSlotFutureManually)
+			.build();
+
+		List<SlotExecutionVertexAssignment> assignments = context.allocateSlotsFor(0, 2);
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+
+		// cancel only one sharing logical slots
+		cancelAction.accept(context, assignments.get(0));
+		assignments = context.allocateSlotsFor(0, 2);
+		// there should be no more physical slot allocations, as the first logical slot reuses the previous shared slot
+		assertThat(context.getSlotProvider().getRequests().keySet(), hasSize(1));
+
+		// cancel all sharing logical slots
+		for (SlotExecutionVertexAssignment assignment : assignments) {
+			cancelAction.accept(context, assignment);
+		}
+		SlotRequestId slotRequestId = context.getSlotProvider().getRequests().keySet().stream().findFirst().get();
+		assertThat(context.getSlotProvider().getCancelations().containsKey(slotRequestId), is(true));
+
+		context.allocateSlotsFor(0, 2);
+		// 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 testPhysicalSlotReleaseLogicalSlots() throws ExecutionException, InterruptedException {
+		AllocationContext context = AllocationContext.newBuilder().setGroups(2).build();
+		List<SlotExecutionVertexAssignment> assignments = context.allocateSlotsFor(0, 2);
+		List<TestingPayload> payloads = assignments
+			.stream()
+			.map(assignment -> {
+				TestingPayload payload = new TestingPayload();
+				assignment.getLogicalSlotFuture().thenAccept(logicalSlot -> logicalSlot.tryAssignPayload(payload));
+				return payload;
+			})
+			.collect(Collectors.toList());
+		TestingPhysicalSlot physicalSlot = context.getSlotProvider().getResponses().values().stream().findFirst().get().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));
+	}
+
+	private static List<ExecutionVertexID> getAssignIds(Collection<SlotExecutionVertexAssignment> assignments) {
+		return assignments.stream().map(SlotExecutionVertexAssignment::getExecutionVertexId).collect(Collectors.toList());
+	}
+
+	private static class AllocationContext {
+		private final List<ExecutionVertexSchedulingRequirements> requirements;
+		private final TestingPhysicalSlotProvider slotProvider;
+		private final TestingSlotSharingStrategy slotSharingStrategy;
+		private final SlotSharingExecutionSlotAllocator allocator;
+		private final TestingSharedSlotProfileRetrieverFactory slotProfileRetrieverFactory;
+
+		AllocationContext(
+				List<ExecutionVertexSchedulingRequirements> requirements,
+				TestingPhysicalSlotProvider slotProvider,
+				TestingSlotSharingStrategy slotSharingStrategy,
+				SlotSharingExecutionSlotAllocator allocator,
+				TestingSharedSlotProfileRetrieverFactory slotProfileRetrieverFactory) {
+			this.requirements = requirements;
+			this.slotProvider = slotProvider;
+			this.slotSharingStrategy = slotSharingStrategy;
+			this.allocator = allocator;
+			this.slotProfileRetrieverFactory = slotProfileRetrieverFactory;
+		}
+
+		public SlotSharingExecutionSlotAllocator getAllocator() {
+			return allocator;
+		}
+
+		private List<SlotExecutionVertexAssignment> allocateSlotsFor(int start, int end) {
+			return allocator.allocateSlotsFor(requirements.subList(start, end));
+		}
+
+		private TestingSlotSharingStrategy getSlotSharingStrategy() {
+			return slotSharingStrategy;
+		}
+
+		private List<ExecutionVertexID> getReqIds(int start, int end) {
+			return getReqIds(requirements.subList(start, end));
+		}
+
+		private TestingPhysicalSlotProvider getSlotProvider() {
+			return slotProvider;
+		}
+
+		private TestingSharedSlotProfileRetrieverFactory getSlotProfileRetrieverFactory() {
+			return slotProfileRetrieverFactory;
+		}
+
+		private static List<ExecutionVertexID> getReqIds(Collection<ExecutionVertexSchedulingRequirements> requirements) {
+			return requirements.stream().map(ExecutionVertexSchedulingRequirements::getExecutionVertexId).collect(Collectors.toList());
+		}
+
+		static Builder newBuilder() {
+			return new Builder();
+		}
+
+		private static class Builder {
+			private int[] groups = { 2, 1 }; // 2 executions in the first group, 1 in the second etc
+			private List<ResourceProfile> resourceProfiles;

Review comment:
       should we explicitly set default values for these fields?
   NPE will be thrown if `resourceProfiles` is null when building the context. For the boolean fields, I feel that explicitly setting them to `false` by default can make it more readable.




----------------------------------------------------------------
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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07fd40233580c1641fda58742e015144d767c2fe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07fd40233580c1641fda58742e015144d767c2fe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5256",
       "triggerID" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5276",
       "triggerID" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 07fd40233580c1641fda58742e015144d767c2fe UNKNOWN
   * 7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5276) 
   * 97ee7a057155f9317a7890d405cb1f2ad363c271 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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/util/DualKeyLinkedMap.java
##########
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.runtime.jobmaster.slotpool;
+package org.apache.flink.util;

Review comment:
       it is in `flink-runtime` but true it should be in `package org.apache.flink.runtime.util;`




----------------------------------------------------------------
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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.SlotProfile;
+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.DualKeyLinkedMap;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Allocates {@link LogicalSlot}s from physical shared slots.
+ *
+ * <p>The allocator maintains a shared slot for each {@link ExecutionSlotSharingGroup}.
+ * It allocates a physical slot for the shared slot and then allocates logical slots from it for scheduled tasks.
+ * The physical slot is lazily allocated for a shared slot, upon any hosted subtask asking for the shared slot.
+ * Each subsequent sharing subtask allocates a logical slot from the existing shared slot. The shared/physical slot
+ * can be released only if all the requested logical slots are released or canceled.
+ */
+class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
+	private static final Logger LOG = LoggerFactory.getLogger(SlotSharingExecutionSlotAllocator.class);
+
+	private final PhysicalSlotProvider slotProvider;
+
+	private final boolean slotWillBeOccupiedIndefinitely;
+
+	private final SlotSharingStrategy slotSharingStrategy;
+
+	private final Map<ExecutionSlotSharingGroup, SharedSlot> sharedSlots;
+
+	private final SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory;
+
+	SlotSharingExecutionSlotAllocator(
+			PhysicalSlotProvider slotProvider,
+			boolean slotWillBeOccupiedIndefinitely,
+			SlotSharingStrategy slotSharingStrategy,
+			SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory) {
+		this.slotProvider = slotProvider;
+		this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+		this.slotSharingStrategy = slotSharingStrategy;
+		this.sharedSlotProfileRetrieverFactory = sharedSlotProfileRetrieverFactory;
+		this.sharedSlots = new HashMap<>();
+	}
+
+	/**
+	 * Creates logical {@link SlotExecutionVertexAssignment}s from physical shared slots.
+	 *
+	 * <p>The allocation has the following steps:
+	 * <ol>
+	 *  <li>Map the executions to {@link ExecutionSlotSharingGroup}s using {@link SlotSharingStrategy}</li>
+	 *  <li>Check which {@link ExecutionSlotSharingGroup}s already have shared slot</li>
+	 *  <li>For all involved {@link ExecutionSlotSharingGroup}s which do not have a shared slot yet:</li>
+	 *  <li>Create a {@link SlotProfile} future using {@link SharedSlotProfileRetriever} and then</li>
+	 *  <li>Allocate a physical slot from the {@link PhysicalSlotProvider}</li>
+	 *  <li>Create a shared slot based on the returned physical slot futures</li>
+	 *  <li>Allocate logical slot futures for the executions from all corresponding shared slots.</li>
+	 *  <li>If a physical slot request fails, associated logical slot requests are canceled within the shared slot</li>
+	 *  <li>Generate {@link SlotExecutionVertexAssignment}s based on the logical slot futures and returns the results.</li>
+	 * </ol>
+	 *
+	 * @param executionVertexSchedulingRequirements the requirements for scheduling the executions.
+	 */
+	@Override
+	public List<SlotExecutionVertexAssignment> allocateSlotsFor(
+			List<ExecutionVertexSchedulingRequirements> executionVertexSchedulingRequirements) {
+		List<ExecutionVertexID> executionVertexIds = executionVertexSchedulingRequirements
+			.stream()
+			.map(ExecutionVertexSchedulingRequirements::getExecutionVertexId)
+			.collect(Collectors.toList());
+
+		SharedSlotProfileRetriever sharedSlotProfileRetriever = sharedSlotProfileRetrieverFactory
+			.createFromBulk(new HashSet<>(executionVertexIds));
+		Map<ExecutionVertexID, SlotExecutionVertexAssignment> assignments = executionVertexIds
+			.stream()
+			.collect(Collectors.groupingBy(slotSharingStrategy::getExecutionSlotSharingGroup))
+			.entrySet()
+			.stream()
+			.flatMap(entry -> allocateLogicalSlotsFromSharedSlot(sharedSlotProfileRetriever, entry.getKey(), entry.getValue()))
+			.collect(Collectors.toMap(SlotExecutionVertexAssignment::getExecutionVertexId, a -> a));
+
+		return executionVertexIds.stream().map(assignments::get).collect(Collectors.toList());
+	}
+
+	@Override
+	public void cancel(ExecutionVertexID executionVertexId) {
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			slotSharingStrategy.getExecutionSlotSharingGroup(executionVertexId);
+		Preconditions.checkNotNull(
+			executionSlotSharingGroup,
+			"There is no ExecutionSlotSharingGroup for ExecutionVertexID " + executionVertexId);
+		SharedSlot slot = sharedSlots.get(executionSlotSharingGroup);
+		if (slot != null) {
+			slot.cancelLogicalSlotRequest(executionVertexId);
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup of ExecutionVertexID {}", executionVertexId);
+		}
+	}
+
+	private Stream<? extends SlotExecutionVertexAssignment> allocateLogicalSlotsFromSharedSlot(
+			SharedSlotProfileRetriever sharedSlotProfileRetriever,
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			Collection<ExecutionVertexID> executions) {
+		SharedSlot sharedSlot = getOrAllocateSharedSlot(executionSlotSharingGroup, sharedSlotProfileRetriever);
+		return executions.stream().map(sharedSlot::createAssignment);
+	}
+
+	private SharedSlot getOrAllocateSharedSlot(
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			SharedSlotProfileRetriever sharedSlotProfileRetriever) {
+		return sharedSlots
+			.computeIfAbsent(executionSlotSharingGroup, group -> {
+				SlotRequestId physicalSlotRequestId = new SlotRequestId();
+				CompletableFuture<PhysicalSlot> physicalSlotFuture = sharedSlotProfileRetriever
+					.getSlotProfileFuture(group)
+					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
+						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
+					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
+				return new SharedSlot(physicalSlotRequestId, group, physicalSlotFuture);
+			});
+	}
+
+	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, SlotExecutionVertexAssignment> 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());
+		}
+
+		SlotExecutionVertexAssignment createAssignment(ExecutionVertexID executionVertexId) {
+			Preconditions.checkArgument(executionSlotSharingGroup.getExecutionVertexIds().contains(executionVertexId));
+			SlotRequestId logicalSlotRequestId = new SlotRequestId();
+			CompletableFuture<LogicalSlot> logicalSlotFuture = allocateLogicalSlot(logicalSlotRequestId, executionVertexId);
+			SlotExecutionVertexAssignment assignment = new SlotExecutionVertexAssignment(executionVertexId, logicalSlotFuture);
+			requestedLogicalSlots.put(
+				executionVertexId,
+				logicalSlotRequestId,
+				assignment);
+			return assignment;
+		}
+
+		private CompletableFuture<LogicalSlot> allocateLogicalSlot(
+				SlotRequestId logicalSlotRequestId,
+				ExecutionVertexID executionVertexId) {
+			String logMessageBase = getLogicalSlotString(logicalSlotRequestId, executionVertexId);

Review comment:
       Maybe add a sanity check here to ensure that the `slotContextFuture` is not completed exceptionally here?
   Otherwise the failed slot future will be registered to `requestedLogicalSlots` cannot be cleared.




----------------------------------------------------------------
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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.SlotProfile;
+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.DualKeyLinkedMap;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
+import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Allocates {@link LogicalSlot}s from physical shared slots.
+ *
+ * <p>The allocator maintains a shared slot for each {@link ExecutionSlotSharingGroup}.
+ * It allocates a physical slot for the shared slot and then allocates logical slots from it for scheduled tasks.
+ * The physical slot is lazily allocated for a shared slot, upon any hosted subtask asking for the shared slot.
+ * Each subsequent sharing subtask allocates a logical slot from the existing shared slot. The shared/physical slot
+ * can be released only if all the requested logical slots are released or canceled.
+ */
+class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
+	private static final Logger LOG = LoggerFactory.getLogger(SlotSharingExecutionSlotAllocator.class);
+
+	private final PhysicalSlotProvider slotProvider;
+
+	private final boolean slotWillBeOccupiedIndefinitely;
+
+	private final SlotSharingStrategy slotSharingStrategy;
+
+	private final Map<ExecutionSlotSharingGroup, SharedSlot> sharedSlots;
+
+	private final SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory;
+
+	SlotSharingExecutionSlotAllocator(
+			PhysicalSlotProvider slotProvider,
+			boolean slotWillBeOccupiedIndefinitely,
+			SlotSharingStrategy slotSharingStrategy,
+			SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory) {
+		this.slotProvider = slotProvider;
+		this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
+		this.slotSharingStrategy = slotSharingStrategy;
+		this.sharedSlotProfileRetrieverFactory = sharedSlotProfileRetrieverFactory;
+		this.sharedSlots = new HashMap<>();
+	}
+
+	/**
+	 * Creates logical {@link SlotExecutionVertexAssignment}s from physical shared slots.
+	 *
+	 * <p>The allocation has the following steps:
+	 * <ol>
+	 *  <li>Map the executions to {@link ExecutionSlotSharingGroup}s using {@link SlotSharingStrategy}</li>
+	 *  <li>Check which {@link ExecutionSlotSharingGroup}s already have shared slot</li>
+	 *  <li>For all involved {@link ExecutionSlotSharingGroup}s which do not have a shared slot yet:</li>
+	 *  <li>Create a {@link SlotProfile} future using {@link SharedSlotProfileRetriever} and then</li>
+	 *  <li>Allocate a physical slot from the {@link PhysicalSlotProvider}</li>
+	 *  <li>Create a shared slot based on the returned physical slot futures</li>
+	 *  <li>Allocate logical slot futures for the executions from all corresponding shared slots.</li>
+	 *  <li>If a physical slot request fails, associated logical slot requests are canceled within the shared slot</li>
+	 *  <li>Generate {@link SlotExecutionVertexAssignment}s based on the logical slot futures and returns the results.</li>
+	 * </ol>
+	 *
+	 * @param executionVertexSchedulingRequirements the requirements for scheduling the executions.
+	 */
+	@Override
+	public List<SlotExecutionVertexAssignment> allocateSlotsFor(
+			List<ExecutionVertexSchedulingRequirements> executionVertexSchedulingRequirements) {
+		List<ExecutionVertexID> executionVertexIds = executionVertexSchedulingRequirements
+			.stream()
+			.map(ExecutionVertexSchedulingRequirements::getExecutionVertexId)
+			.collect(Collectors.toList());
+
+		SharedSlotProfileRetriever sharedSlotProfileRetriever = sharedSlotProfileRetrieverFactory
+			.createFromBulk(new HashSet<>(executionVertexIds));
+		Map<ExecutionVertexID, SlotExecutionVertexAssignment> assignments = executionVertexIds
+			.stream()
+			.collect(Collectors.groupingBy(slotSharingStrategy::getExecutionSlotSharingGroup))
+			.entrySet()
+			.stream()
+			.flatMap(entry -> allocateLogicalSlotsFromSharedSlot(sharedSlotProfileRetriever, entry.getKey(), entry.getValue()))
+			.collect(Collectors.toMap(SlotExecutionVertexAssignment::getExecutionVertexId, a -> a));
+
+		return executionVertexIds.stream().map(assignments::get).collect(Collectors.toList());
+	}
+
+	@Override
+	public void cancel(ExecutionVertexID executionVertexId) {
+		ExecutionSlotSharingGroup executionSlotSharingGroup =
+			slotSharingStrategy.getExecutionSlotSharingGroup(executionVertexId);
+		Preconditions.checkNotNull(
+			executionSlotSharingGroup,
+			"There is no ExecutionSlotSharingGroup for ExecutionVertexID " + executionVertexId);
+		SharedSlot slot = sharedSlots.get(executionSlotSharingGroup);
+		if (slot != null) {
+			slot.cancelLogicalSlotRequest(executionVertexId);
+		} else {
+			LOG.debug("There is no slot for ExecutionSlotSharingGroup of ExecutionVertexID {}", executionVertexId);
+		}
+	}
+
+	private Stream<? extends SlotExecutionVertexAssignment> allocateLogicalSlotsFromSharedSlot(
+			SharedSlotProfileRetriever sharedSlotProfileRetriever,
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			Collection<ExecutionVertexID> executions) {
+		SharedSlot sharedSlot = getOrAllocateSharedSlot(executionSlotSharingGroup, sharedSlotProfileRetriever);
+		return executions.stream().map(sharedSlot::createAssignment);
+	}
+
+	private SharedSlot getOrAllocateSharedSlot(
+			ExecutionSlotSharingGroup executionSlotSharingGroup,
+			SharedSlotProfileRetriever sharedSlotProfileRetriever) {
+		return sharedSlots
+			.computeIfAbsent(executionSlotSharingGroup, group -> {
+				SlotRequestId physicalSlotRequestId = new SlotRequestId();
+				CompletableFuture<PhysicalSlot> physicalSlotFuture = sharedSlotProfileRetriever
+					.getSlotProfileFuture(group)
+					.thenCompose(slotProfile -> slotProvider.allocatePhysicalSlot(
+						new PhysicalSlotRequest(physicalSlotRequestId, slotProfile, slotWillBeOccupiedIndefinitely)))
+					.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
+				return new SharedSlot(physicalSlotRequestId, group, physicalSlotFuture);
+			});
+	}
+
+	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, SlotExecutionVertexAssignment> 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());
+		}
+
+		SlotExecutionVertexAssignment createAssignment(ExecutionVertexID executionVertexId) {
+			Preconditions.checkArgument(executionSlotSharingGroup.getExecutionVertexIds().contains(executionVertexId));
+			SlotRequestId logicalSlotRequestId = new SlotRequestId();
+			CompletableFuture<LogicalSlot> logicalSlotFuture = allocateLogicalSlot(logicalSlotRequestId, executionVertexId);
+			SlotExecutionVertexAssignment assignment = new SlotExecutionVertexAssignment(executionVertexId, logicalSlotFuture);
+			requestedLogicalSlots.put(
+				executionVertexId,
+				logicalSlotRequestId,
+				assignment);
+			return assignment;
+		}
+
+		private CompletableFuture<LogicalSlot> allocateLogicalSlot(
+				SlotRequestId logicalSlotRequestId,
+				ExecutionVertexID executionVertexId) {
+			String logMessageBase = getLogicalSlotString(logicalSlotRequestId, executionVertexId);
+			LOG.debug("Request a {}", logMessageBase);
+			CompletableFuture<LogicalSlot> logicalSlotFuture = slotContextFuture.thenApply(physicalSlot -> {
+				LOG.debug("Allocated {}", logMessageBase);
+				return new SingleLogicalSlot(
+					logicalSlotRequestId,
+					physicalSlot,
+					null,
+					Locality.UNKNOWN,
+					this,
+					slotWillBeOccupiedIndefinitely);
+			});
+			logicalSlotFuture.exceptionally(cause -> {
+				LOG.debug("Failed {}", logMessageBase);
+				cancelLogicalSlotRequest(logicalSlotRequestId);
+				return null;
+			});
+			return logicalSlotFuture;
+		}
+
+		void cancelLogicalSlotRequest(ExecutionVertexID executionVertexID) {
+			cancelLogicalSlotRequest(requestedLogicalSlots.getKeyBByKeyA(executionVertexID));
+		}
+
+		void cancelLogicalSlotRequest(SlotRequestId logicalSlotRequestId) {
+			SlotExecutionVertexAssignment assignment = requestedLogicalSlots.getValueByKeyB(logicalSlotRequestId);
+			if (assignment != null) {
+				LOG.debug("Cancel {}", getLogicalSlotString(logicalSlotRequestId, assignment.getExecutionVertexId()));
+				assignment.getLogicalSlotFuture().cancel(false);
+				requestedLogicalSlots.removeKeyB(logicalSlotRequestId);
+			} else {
+				LOG.debug("No SlotExecutionVertexAssignment for logical {} from physical %s", logicalSlotRequestId);
+			}
+			if (requestedLogicalSlots.values().isEmpty()) {
+				sharedSlots.remove(executionSlotSharingGroup);
+				slotProvider.cancelSlotRequest(
+					physicalSlotRequestId,
+					new FlinkException("Slot is being returned from SlotSharingExecutionSlotAllocator."));
+			}
+		}
+
+		@Override
+		public void returnLogicalSlot(LogicalSlot logicalSlot) {
+			cancelLogicalSlotRequest(logicalSlot.getSlotRequestId());
+		}
+
+		@Override
+		public void release(Throwable cause) {
+			for (SlotExecutionVertexAssignment assignment : requestedLogicalSlots.values()) {
+				ExecutionVertexID executionVertexId = assignment.getExecutionVertexId();
+				LOG.debug("Release {}", getLogicalSlotString(requestedLogicalSlots.getKeyBByKeyA(executionVertexId), executionVertexId));
+				assignment

Review comment:
       When `SharedSlot#release(...)` is invoked, it indicates that the physical slot was successfully allocated, otherwise the shared slot is not assigned as the payload yet. This further means the logical slot futures have also completed normally. This is how I understand the assumption.
   
   If we do not have the assumption the logical slot futures have completed normally, we need to cancel the `slotContextFuture` as well as all the pending logical slot futures in this `release()` 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] flinkbot edited a comment on pull request #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07fd40233580c1641fda58742e015144d767c2fe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07fd40233580c1641fda58742e015144d767c2fe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5256",
       "triggerID" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5276",
       "triggerID" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5290",
       "triggerID" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0f6489c64935c086c4cc980bc7f5cf858f8544c4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 07fd40233580c1641fda58742e015144d767c2fe UNKNOWN
   * 97ee7a057155f9317a7890d405cb1f2ad363c271 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5290) 
   * 0f6489c64935c086c4cc980bc7f5cf858f8544c4 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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5212",
       "triggerID" : "8a735691f03d9abd842d7917303ccf9c9fe21e56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2233727954b8a6272c4a772a03bccc08b5e97720",
       "triggerType" : "PUSH"
     }, {
       "hash" : "708450b388222f8c266604429e604327bce83491",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5242",
       "triggerID" : "708450b388222f8c266604429e604327bce83491",
       "triggerType" : "PUSH"
     }, {
       "hash" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5251",
       "triggerID" : "223e7673843cdc4fc6467ce0ed9ad0ff0630aef5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07fd40233580c1641fda58742e015144d767c2fe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07fd40233580c1641fda58742e015144d767c2fe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5256",
       "triggerID" : "2d0c93809d722ca1325ebec8aa4de38c83cc3f5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5276",
       "triggerID" : "7a70b16f8bed0bd3e18df37ef11e05612eaa3f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5290",
       "triggerID" : "97ee7a057155f9317a7890d405cb1f2ad363c271",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2233727954b8a6272c4a772a03bccc08b5e97720 UNKNOWN
   * 07fd40233580c1641fda58742e015144d767c2fe UNKNOWN
   * 97ee7a057155f9317a7890d405cb1f2ad363c271 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5290) 
   
   <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 #13071: [FLINK-18751][Coordination] Implement SlotSharingExecutionSlotAllocator

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/DualKeyLinkedMap.java
##########
@@ -43,63 +43,48 @@
  * @param <B> Type of key B. Key B is the secondary key.
  * @param <V> Type of the value
  */
-class DualKeyLinkedMap<A, B, V> {

Review comment:
       There is a typo in the commit message:
   `[hotfrix]` -> `[hotfix]`




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