Skip to content
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.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 6 additions & 0 deletions docs/layouts/shortcodes/generated/all_jobmanager_section.html
Original file line number Diff line number Diff line change
Expand Up @@ -86,5 +86,11 @@
<td>Integer</td>
<td>The maximum number of failures collected by the exception history per job.</td>
</tr>
<tr>
<td><h5>jobmanager.scheduler.enable-slot-allocate-order-optimization</h5></td>
<td style="word-wrap: break-word;">false</td>
<td>Boolean</td>
<td>Flag indicating whether to enable the slot allocate optimization for DefaultScheduler and AdaptiveScheduler. It could distribute subtasks (belong to the same operator) across as many TaskManagers as possible.</td>
</tr>
</tbody>
</table>
Original file line number Diff line number Diff line change
Expand Up @@ -80,5 +80,11 @@
<td>Integer</td>
<td>Defines the maximum number of slots that the Flink cluster allocates. This configuration option is meant for limiting the resource consumption for batch workloads. It is not recommended to configure this option for streaming workloads, which may fail if there are not enough slots. Note that this configuration option does not take effect for standalone clusters, where how many slots are allocated is not controlled by Flink.</td>
</tr>
<tr>
<td><h5>jobmanager.scheduler.enable-slot-allocate-order-optimization</h5></td>
<td style="word-wrap: break-word;">false</td>
<td>Boolean</td>
<td>Flag indicating whether to enable the slot allocate optimization for DefaultScheduler and AdaptiveScheduler. It could distribute subtasks (belong to the same operator) across as many TaskManagers as possible.</td>
</tr>
</tbody>
</table>
Original file line number Diff line number Diff line change
Expand Up @@ -164,5 +164,11 @@
<td>Long</td>
<td>The timeout in milliseconds for requesting a slot from Slot Pool.</td>
</tr>
<tr>
<td><h5>jobmanager.scheduler.enable-slot-allocate-order-optimization</h5></td>
<td style="word-wrap: break-word;">false</td>
<td>Boolean</td>
<td>Flag indicating whether to enable the slot allocate optimization for DefaultScheduler and AdaptiveScheduler. It could distribute subtasks (belong to the same operator) across as many TaskManagers as possible.</td>
</tr>
</tbody>
</table>
Original file line number Diff line number Diff line change
Expand Up @@ -454,6 +454,18 @@ public enum SchedulerType {
.withDescription(
"Controls whether partitions should already be released during the job execution.");

@Documentation.Section({
Documentation.Sections.EXPERT_SCHEDULING,
Documentation.Sections.ALL_JOB_MANAGER
})
public static final ConfigOption<Boolean> SLOT_ALLOCATE_ORDER_OPTIMIZATION =
key("jobmanager.scheduler.enable-slot-allocate-order-optimization")
.booleanType()
.defaultValue(false)
.withDescription(
"Flag indicating whether to enable the slot allocate optimization for DefaultScheduler and AdaptiveScheduler. "
+ "It could distribute subtasks (belong to the same operator) across as many TaskManagers as possible.");

// ---------------------------------------------------------------------------------------------

private JobManagerOptions() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.flink.configuration.CheckpointingOptions;
import org.apache.flink.configuration.ClusterOptions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
import org.apache.flink.runtime.jobgraph.JobType;
import org.apache.flink.runtime.jobmaster.slotpool.LocationPreferenceSlotSelectionStrategy;
Expand Down Expand Up @@ -100,12 +101,16 @@ private static DefaultSchedulerComponents createPipelinedRegionSchedulerComponen
slotPool, SystemClock.getInstance());
final PhysicalSlotProvider physicalSlotProvider =
new PhysicalSlotProviderImpl(slotSelectionStrategy, slotPool);
final boolean enableSlotAllocateOrderOptimization =
jobMasterConfiguration.getBoolean(
JobManagerOptions.SLOT_ALLOCATE_ORDER_OPTIMIZATION);
final ExecutionSlotAllocatorFactory allocatorFactory =
new SlotSharingExecutionSlotAllocatorFactory(
physicalSlotProvider,
jobType == JobType.STREAMING,
bulkChecker,
slotRequestTimeout);
slotRequestTimeout,
enableSlotAllocateOrderOptimization);
return new DefaultSchedulerComponents(
new PipelinedRegionSchedulingStrategy.Factory(),
bulkChecker::start,
Expand Down
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.jobgraph.JobVertexID;
import org.apache.flink.runtime.util.CircleIterator;

import java.util.ArrayList;
import java.util.Collection;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;

/** A default implementation of {@link SlotSharingGroupOrderFunction}. */
public class EvenlySlotSharingGroupOrderStrategy implements SlotSharingGroupOrderFunction {

@Override
public List<ExecutionSlotSharingGroup> determineOrder(
Collection<ExecutionSlotSharingGroup> groups) {
final Map<Set<JobVertexID>, List<ExecutionSlotSharingGroup>> slotSharingGroups =
groups.stream()
.collect(Collectors.groupingBy(ExecutionSlotSharingGroup::getJobVertexIds));
final Comparator<Set<JobVertexID>> comparator = Comparator.comparingInt(Collection::size);

final CircleIterator<Set<JobVertexID>, ExecutionSlotSharingGroup> iterator =
new CircleIterator<>(slotSharingGroups, comparator.reversed());
List<ExecutionSlotSharingGroup> orders = new ArrayList<>();
while (iterator.hasNext()) {
orders.add(iterator.next());
}
return orders;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,12 +19,14 @@
package org.apache.flink.runtime.scheduler;

import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
import org.apache.flink.util.Preconditions;

import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import java.util.stream.Collectors;

/** Represents execution vertices that will run the same shared slot. */
class ExecutionSlotSharingGroup {
Expand Down Expand Up @@ -52,4 +54,10 @@ ResourceProfile getResourceProfile() {
Set<ExecutionVertexID> getExecutionVertexIds() {
return Collections.unmodifiableSet(executionVertexIds);
}

Set<JobVertexID> getJobVertexIds() {
return executionVertexIds.stream()
.map(ExecutionVertexID::getJobVertexId)
.collect(Collectors.toSet());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -76,14 +76,17 @@ class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {

private final Function<ExecutionVertexID, ResourceProfile> resourceProfileRetriever;

private final SlotSharingGroupOrderFunction slotSharingGroupOrderFunction;

SlotSharingExecutionSlotAllocator(
PhysicalSlotProvider slotProvider,
boolean slotWillBeOccupiedIndefinitely,
SlotSharingStrategy slotSharingStrategy,
SharedSlotProfileRetrieverFactory sharedSlotProfileRetrieverFactory,
PhysicalSlotRequestBulkChecker bulkChecker,
Time allocationTimeout,
Function<ExecutionVertexID, ResourceProfile> resourceProfileRetriever) {
Function<ExecutionVertexID, ResourceProfile> resourceProfileRetriever,
SlotSharingGroupOrderFunction slotSharingGroupOrderFunction) {
this.slotProvider = checkNotNull(slotProvider);
this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
this.slotSharingStrategy = checkNotNull(slotSharingStrategy);
Expand All @@ -92,6 +95,7 @@ class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
this.allocationTimeout = checkNotNull(allocationTimeout);
this.resourceProfileRetriever = checkNotNull(resourceProfileRetriever);
this.sharedSlots = new IdentityHashMap<>();
this.slotSharingGroupOrderFunction = checkNotNull(slotSharingGroupOrderFunction);
}

/**
Expand Down Expand Up @@ -128,8 +132,10 @@ public List<SlotExecutionVertexAssignment> allocateSlotsFor(
.collect(
Collectors.groupingBy(
slotSharingStrategy::getExecutionSlotSharingGroup));
List<ExecutionSlotSharingGroup> slotSharingGroups =
slotSharingGroupOrderFunction.determineOrder(executionsByGroup.keySet());
Map<ExecutionSlotSharingGroup, SharedSlot> slots =
executionsByGroup.keySet().stream()
slotSharingGroups.stream()
.map(group -> getOrAllocateSharedSlot(group, sharedSlotProfileRetriever))
.collect(
Collectors.toMap(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,16 @@
import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequestBulkChecker;
import org.apache.flink.runtime.scheduler.SharedSlotProfileRetriever.SharedSlotProfileRetrieverFactory;

import java.util.ArrayList;

/** Factory for {@link SlotSharingExecutionSlotAllocator}. */
class SlotSharingExecutionSlotAllocatorFactory implements ExecutionSlotAllocatorFactory {
private final PhysicalSlotProvider slotProvider;

private final boolean slotWillBeOccupiedIndefinitely;

private final boolean enableSlotAllocateOrderOptimization;

private final PhysicalSlotRequestBulkChecker bulkChecker;

private final Time allocationTimeout;
Expand All @@ -39,26 +43,30 @@ class SlotSharingExecutionSlotAllocatorFactory implements ExecutionSlotAllocator
PhysicalSlotProvider slotProvider,
boolean slotWillBeOccupiedIndefinitely,
PhysicalSlotRequestBulkChecker bulkChecker,
Time allocationTimeout) {
Time allocationTimeout,
boolean enableSlotAllocateOrderOptimization) {
this(
slotProvider,
slotWillBeOccupiedIndefinitely,
bulkChecker,
allocationTimeout,
new LocalInputPreferredSlotSharingStrategy.Factory());
new LocalInputPreferredSlotSharingStrategy.Factory(),
enableSlotAllocateOrderOptimization);
}

SlotSharingExecutionSlotAllocatorFactory(
PhysicalSlotProvider slotProvider,
boolean slotWillBeOccupiedIndefinitely,
PhysicalSlotRequestBulkChecker bulkChecker,
Time allocationTimeout,
SlotSharingStrategy.Factory slotSharingStrategyFactory) {
SlotSharingStrategy.Factory slotSharingStrategyFactory,
boolean enableSlotAllocateOrderOptimization) {
this.slotProvider = slotProvider;
this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
this.bulkChecker = bulkChecker;
this.slotSharingStrategyFactory = slotSharingStrategyFactory;
this.allocationTimeout = allocationTimeout;
this.enableSlotAllocateOrderOptimization = enableSlotAllocateOrderOptimization;
}

@Override
Expand All @@ -80,6 +88,9 @@ public ExecutionSlotAllocator createInstance(final ExecutionSlotAllocationContex
sharedSlotProfileRetrieverFactory,
bulkChecker,
allocationTimeout,
context::getResourceProfile);
context::getResourceProfile,
enableSlotAllocateOrderOptimization
? new EvenlySlotSharingGroupOrderStrategy()
: ArrayList::new);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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 java.util.Collection;
import java.util.List;

/** A function for determining ExecutionSlotSharingGroup allocate orders. */
@FunctionalInterface
public interface SlotSharingGroupOrderFunction {
/**
* Sort the groups to control the allocate order.
*
* @param groups which need to be sorted
* @return sorted groups
*/
List<ExecutionSlotSharingGroup> determineOrder(Collection<ExecutionSlotSharingGroup> groups);
}
Original file line number Diff line number Diff line change
Expand Up @@ -38,12 +38,14 @@
import org.apache.flink.runtime.scheduler.ExecutionGraphFactory;
import org.apache.flink.runtime.scheduler.SchedulerNG;
import org.apache.flink.runtime.scheduler.SchedulerNGFactory;
import org.apache.flink.runtime.scheduler.adaptive.allocator.EvenlySlotSharingOrderStrategy;
import org.apache.flink.runtime.scheduler.adaptive.allocator.SlotSharingSlotAllocator;
import org.apache.flink.runtime.shuffle.ShuffleMaster;

import org.slf4j.Logger;

import java.time.Duration;
import java.util.ArrayList;
import java.util.concurrent.Executor;
import java.util.concurrent.ScheduledExecutorService;

Expand Down Expand Up @@ -103,7 +105,7 @@ public SchedulerNG createInstance(
jobGraph.getJobID());

final SlotSharingSlotAllocator slotAllocator =
createSlotSharingSlotAllocator(declarativeSlotPool);
createSlotSharingSlotAllocator(declarativeSlotPool, jobMasterConfiguration);

final ExecutionGraphFactory executionGraphFactory =
new DefaultExecutionGraphFactory(
Expand Down Expand Up @@ -143,10 +145,17 @@ public JobManagerOptions.SchedulerType getSchedulerType() {
}

public static SlotSharingSlotAllocator createSlotSharingSlotAllocator(
DeclarativeSlotPool declarativeSlotPool) {
DeclarativeSlotPool declarativeSlotPool, Configuration jobMasterConfiguration) {

final boolean enableSlotAllocateOrderOptimization =
jobMasterConfiguration.getBoolean(
JobManagerOptions.SLOT_ALLOCATE_ORDER_OPTIMIZATION);
return SlotSharingSlotAllocator.createSlotSharingSlotAllocator(
declarativeSlotPool::reserveFreeSlot,
declarativeSlotPool::freeReservedSlot,
declarativeSlotPool::containsFreeSlot);
declarativeSlotPool::containsFreeSlot,
enableSlotAllocateOrderOptimization
? new EvenlySlotSharingOrderStrategy()
: ArrayList::new);
}
}
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.adaptive.allocator;

import org.apache.flink.runtime.jobmaster.SlotInfo;
import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
import org.apache.flink.runtime.util.CircleIterator;

import java.util.ArrayList;
import java.util.Collection;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;

/** A default implementation of {@link SlotOrderFunction}. */
public class EvenlySlotSharingOrderStrategy implements SlotOrderFunction {

@Override
public List<? extends SlotInfo> determineOrder(Collection<? extends SlotInfo> slots) {
final Map<TaskManagerLocation, List<SlotInfo>> taskManagerSlots =
slots.stream().collect(Collectors.groupingBy(SlotInfo::getTaskManagerLocation));
final Comparator<TaskManagerLocation> comparator =
Comparator.comparingInt(taskManager -> taskManagerSlots.get(taskManager).size());

final CircleIterator<TaskManagerLocation, SlotInfo> iterator =
new CircleIterator<>(taskManagerSlots, comparator.reversed());
List<SlotInfo> orders = new ArrayList<>();
while (iterator.hasNext()) {
orders.add(iterator.next());
}

return orders;
}
}
Loading