Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-19552][coordination] Consider only available input location preferences for slot profile in pipelined region scheduling #13730

Closed
wants to merge 5 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
/*
* 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.scheduler.strategy.ExecutionVertexID;
import org.apache.flink.runtime.taskmanager.TaskManagerLocation;

import java.util.Collection;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;

/**
* {@link InputsLocationsRetriever} which filters out only already available input locations.
*/
class AvailableInputsLocationsRetriever implements InputsLocationsRetriever {
private final InputsLocationsRetriever inputsLocationsRetriever;

AvailableInputsLocationsRetriever(InputsLocationsRetriever inputsLocationsRetriever) {
this.inputsLocationsRetriever = inputsLocationsRetriever;
}

@Override
public Collection<Collection<ExecutionVertexID>> getConsumedResultPartitionsProducers(
ExecutionVertexID executionVertexId) {
return inputsLocationsRetriever.getConsumedResultPartitionsProducers(executionVertexId);
}

@Override
public Optional<CompletableFuture<TaskManagerLocation>> getTaskManagerLocation(ExecutionVertexID executionVertexId) {
return inputsLocationsRetriever
.getTaskManagerLocation(executionVertexId)
.filter(future -> future.isDone() && !future.isCompletedExceptionally());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,8 @@ class DefaultExecutionSlotAllocatorFactory implements ExecutionSlotAllocatorFact

@Override
public ExecutionSlotAllocator createInstance(final ExecutionSlotAllocationContext context) {
return new DefaultExecutionSlotAllocator(slotProvider, context::getPreferredLocations);
PreferredLocationsRetriever preferredLocationsRetriever =
new DefaultPreferredLocationsRetriever(context, context);
return new DefaultExecutionSlotAllocator(slotProvider, preferredLocationsRetriever);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,7 @@
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
Expand Down Expand Up @@ -518,13 +519,6 @@ private void notifyCoordinatorOfCancellation(ExecutionVertex vertex) {

private class DefaultExecutionSlotAllocationContext implements ExecutionSlotAllocationContext {

@Override
public CompletableFuture<Collection<TaskManagerLocation>> getPreferredLocations(
final ExecutionVertexID executionVertexId,
final Set<ExecutionVertexID> producersToIgnore) {
return getPreferredLocationsRetriever().getPreferredLocations(executionVertexId, producersToIgnore);
}

@Override
public ResourceProfile getResourceProfile(final ExecutionVertexID executionVertexId) {
return getExecutionVertex(executionVertexId).getResourceProfile();
Expand All @@ -549,5 +543,22 @@ public Set<SlotSharingGroup> getLogicalSlotSharingGroups() {
public Set<CoLocationGroupDesc> getCoLocationGroups() {
return getJobGraph().getCoLocationGroupDescriptors();
}

@Override
public Collection<Collection<ExecutionVertexID>> getConsumedResultPartitionsProducers(
ExecutionVertexID executionVertexId) {
return inputsLocationsRetriever.getConsumedResultPartitionsProducers(executionVertexId);
}

@Override
public Optional<CompletableFuture<TaskManagerLocation>> getTaskManagerLocation(
ExecutionVertexID executionVertexId) {
return inputsLocationsRetriever.getTaskManagerLocation(executionVertexId);
}

@Override
public Optional<TaskManagerLocation> getStateLocation(ExecutionVertexID executionVertexId) {
return stateLocationRetriever.getStateLocation(executionVertexId);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* 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.scheduler.strategy.ExecutionVertexID;
import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
import org.apache.flink.util.Preconditions;

import java.util.Collection;
import java.util.Set;
import java.util.concurrent.CompletableFuture;

/**
* Synchronous version of {@link DefaultPreferredLocationsRetriever}.
*
* <p>This class turns {@link DefaultPreferredLocationsRetriever} into {@link SyncPreferredLocationsRetriever}.
* The method {@link #getPreferredLocations(ExecutionVertexID, Set)} does not return {@link CompletableFuture}
* of preferred locations, it returns only locations which are available immediately.
* This behaviour is achieved by wrapping the original {@link InputsLocationsRetriever} with
* {@link AvailableInputsLocationsRetriever} and hence making it synchronous without blocking.
* As {@link StateLocationRetriever} is already synchronous,
* the overall location retrieval becomes synchronous without blocking.
*/
class DefaultSyncPreferredLocationsRetriever implements SyncPreferredLocationsRetriever {
private final PreferredLocationsRetriever asyncPreferredLocationsRetriever;

DefaultSyncPreferredLocationsRetriever(
StateLocationRetriever stateLocationRetriever,
InputsLocationsRetriever inputsLocationsRetriever) {
this.asyncPreferredLocationsRetriever = new DefaultPreferredLocationsRetriever(
stateLocationRetriever,
new AvailableInputsLocationsRetriever(inputsLocationsRetriever));
}

@Override
public Collection<TaskManagerLocation> getPreferredLocations(
ExecutionVertexID executionVertexId,
Set<ExecutionVertexID> producersToIgnore) {
CompletableFuture<Collection<TaskManagerLocation>> preferredLocationsFuture =
asyncPreferredLocationsRetriever.getPreferredLocations(executionVertexId, producersToIgnore);
Preconditions.checkState(preferredLocationsFuture.isDone());
// it is safe to do the blocking call here
// as the underlying InputsLocationsRetriever returns only immediately available locations
return preferredLocationsFuture.join();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@
/**
* Context for slot allocation.
*/
interface ExecutionSlotAllocationContext extends PreferredLocationsRetriever {
interface ExecutionSlotAllocationContext extends InputsLocationsRetriever, StateLocationRetriever {

/**
* Returns required resources for an execution vertex.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
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.concurrent.FutureUtils;
import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
import org.apache.flink.util.Preconditions;
Expand All @@ -31,20 +30,19 @@
import java.util.HashSet;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.function.Function;
import java.util.stream.Collectors;

/**
* Factory for {@link MergingSharedSlotProfileRetriever}.
*/
class MergingSharedSlotProfileRetrieverFactory implements SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory {
private final PreferredLocationsRetriever preferredLocationsRetriever;
private final SyncPreferredLocationsRetriever preferredLocationsRetriever;

private final Function<ExecutionVertexID, AllocationID> priorAllocationIdRetriever;

MergingSharedSlotProfileRetrieverFactory(
PreferredLocationsRetriever preferredLocationsRetriever,
SyncPreferredLocationsRetriever preferredLocationsRetriever,
Function<ExecutionVertexID, AllocationID> priorAllocationIdRetriever) {
this.preferredLocationsRetriever = Preconditions.checkNotNull(preferredLocationsRetriever);
this.priorAllocationIdRetriever = Preconditions.checkNotNull(priorAllocationIdRetriever);
Expand Down Expand Up @@ -96,33 +94,24 @@ private MergingSharedSlotProfileRetriever(
*
* @param executionSlotSharingGroup executions sharing the slot.
* @param physicalSlotResourceProfile {@link ResourceProfile} of the slot.
* @return a future of the {@link SlotProfile} to allocate for the {@code executionSlotSharingGroup}.
* @return {@link SlotProfile} to allocate for the {@code executionSlotSharingGroup}.
*/
@Override
public CompletableFuture<SlotProfile> getSlotProfileFuture(
public SlotProfile getSlotProfile(
ExecutionSlotSharingGroup executionSlotSharingGroup,
ResourceProfile physicalSlotResourceProfile) {
Collection<AllocationID> priorAllocations = new HashSet<>();
Collection<CompletableFuture<Collection<TaskManagerLocation>>> preferredLocationsPerExecution = new ArrayList<>();
Collection<TaskManagerLocation> preferredLocations = new ArrayList<>();
for (ExecutionVertexID execution : executionSlotSharingGroup.getExecutionVertexIds()) {
priorAllocations.add(priorAllocationIdRetriever.apply(execution));
preferredLocationsPerExecution.add(preferredLocationsRetriever
.getPreferredLocations(execution, producersToIgnore));
preferredLocations.addAll(preferredLocationsRetriever.getPreferredLocations(execution, producersToIgnore));
}

CompletableFuture<Collection<TaskManagerLocation>> preferredLocationsFuture = FutureUtils
.combineAll(preferredLocationsPerExecution)
.thenApply(executionPreferredLocations ->
executionPreferredLocations.stream().flatMap(Collection::stream).collect(Collectors.toList()));

return preferredLocationsFuture.thenApply(
preferredLocations ->
SlotProfile.priorAllocation(
physicalSlotResourceProfile,
physicalSlotResourceProfile,
preferredLocations,
priorAllocations,
allBulkPriorAllocationIds));
return SlotProfile.priorAllocation(
physicalSlotResourceProfile,
physicalSlotResourceProfile,
preferredLocations,
priorAllocations,
allBulkPriorAllocationIds);
}
}
}