Skip to content
This repository was archived by the owner on Mar 31, 2023. It is now read-only.

Commit 5f49f98

Browse files
authored
Minor improvements (#177)
* Add task batch size in order to dynamically increase/reduce scheduling iteration time * Name the fenzo threads
1 parent e462045 commit 5f49f98

File tree

4 files changed

+129
-2
lines changed

4 files changed

+129
-2
lines changed

fenzo-core/src/main/java/com/netflix/fenzo/TaskScheduler.java

+26-1
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@
1616

1717
package com.netflix.fenzo;
1818

19+
import com.netflix.fenzo.common.ThreadFactoryBuilder;
1920
import com.netflix.fenzo.plugins.NoOpScaleDownOrderEvaluator;
2021
import com.netflix.fenzo.queues.Assignable;
2122
import com.netflix.fenzo.queues.QueuableTask;
@@ -34,8 +35,10 @@
3435
import java.util.concurrent.ExecutorService;
3536
import java.util.concurrent.Executors;
3637
import java.util.concurrent.Future;
38+
import java.util.concurrent.ThreadFactory;
3739
import java.util.concurrent.atomic.AtomicBoolean;
3840
import java.util.concurrent.atomic.AtomicInteger;
41+
import java.util.function.Supplier;
3942
import java.util.stream.Collectors;
4043

4144
/**
@@ -103,6 +106,7 @@ public Boolean call(Double f) {
103106
private boolean singleOfferMode=false;
104107
private final List<SchedulingEventListener> schedulingEventListeners = new ArrayList<>();
105108
private int maxConcurrent = Runtime.getRuntime().availableProcessors();
109+
private Supplier<Long> taskBatchSizeSupplier = () -> Long.MAX_VALUE;
106110

107111
/**
108112
* (Required) Call this method to establish a method that your task scheduler will call to notify you
@@ -442,6 +446,20 @@ public Builder withMaxConcurrent(int maxConcurrent) {
442446
return this;
443447
}
444448

449+
/**
450+
* Use the given supplier to determine how many successful tasks should be evaluated in the next scheduling iteration. This
451+
* can be used to dynamically change how many successful task evaluations are done in order to increase/reduce the scheduling iteration
452+
* duration. The default supplier implementation will return {@link Long#MAX_VALUE} such that all tasks will be
453+
* evaluated.
454+
*
455+
* @param taskBatchSizeSupplier the supplier that returns the task batch size for the next scheduling iteration.
456+
* @return this same {@code Builder}, suitable for further chaining or to build the {@link TaskSchedulingService}.
457+
*/
458+
public Builder withTaskBatchSizeSupplier(Supplier<Long> taskBatchSizeSupplier) {
459+
this.taskBatchSizeSupplier = taskBatchSizeSupplier;
460+
return this;
461+
}
462+
445463
/**
446464
* Creates a {@link TaskScheduler} based on the various builder methods you have chained.
447465
*
@@ -497,7 +515,8 @@ private TaskScheduler(Builder builder) {
497515
throw new IllegalArgumentException("Lease reject action must be non-null");
498516
this.builder = builder;
499517
this.maxConcurrent = builder.maxConcurrent;
500-
this.executorService = Executors.newFixedThreadPool(maxConcurrent);
518+
ThreadFactory threadFactory = ThreadFactoryBuilder.newBuilder().withNameFormat("fenzo-worker-%d").build();
519+
this.executorService = Executors.newFixedThreadPool(maxConcurrent, threadFactory);
501520
this.stateMonitor = new StateMonitor();
502521
this.schedulingEventListener = CompositeSchedulingEventListener.of(builder.schedulingEventListeners);
503522
taskTracker = new TaskTracker();
@@ -795,6 +814,8 @@ private SchedulingResult doSchedule(
795814
Set<TaskRequest> failedTasksForAutoScaler = new HashSet<>();
796815
Map<String, VMAssignmentResult> resultMap = new HashMap<>(avms.size());
797816
final SchedulingResult schedulingResult = new SchedulingResult(resultMap);
817+
long taskBatchSize = builder.taskBatchSizeSupplier.get();
818+
long tasksIterationCount = 0;
798819
if(avms.isEmpty()) {
799820
while (true) {
800821
final Assignable<? extends TaskRequest> taskOrFailure = taskIterator.next();
@@ -806,6 +827,9 @@ private SchedulingResult doSchedule(
806827
schedulingEventListener.onScheduleStart();
807828
try {
808829
while (true) {
830+
if (tasksIterationCount >= taskBatchSize) {
831+
break;
832+
}
809833
final Assignable<? extends TaskRequest> taskOrFailure = taskIterator.next();
810834
if(logger.isDebugEnabled())
811835
logger.debug("TaskSched: task=" + (taskOrFailure == null? "null" : taskOrFailure.getTask().getId()));
@@ -904,6 +928,7 @@ public EvalResult call() throws Exception {
904928
logger.debug("Task {}: found successful assignment on host {}", task.getId(),
905929
successfulResult.getHostname());
906930
successfulResult.assignResult();
931+
tasksIterationCount++;
907932
failedTasksForAutoScaler.remove(task);
908933
schedulingEventListener.onAssignment(successfulResult);
909934
}

fenzo-core/src/main/java/com/netflix/fenzo/TaskSchedulingService.java

+4-1
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@
1616

1717
package com.netflix.fenzo;
1818

19+
import com.netflix.fenzo.common.ThreadFactoryBuilder;
1920
import com.netflix.fenzo.functions.Action0;
2021
import com.netflix.fenzo.functions.Action1;
2122
import com.netflix.fenzo.functions.Func1;
@@ -39,6 +40,7 @@
3940
import java.util.concurrent.LinkedBlockingQueue;
4041
import java.util.concurrent.ScheduledExecutorService;
4142
import java.util.concurrent.ScheduledThreadPoolExecutor;
43+
import java.util.concurrent.ThreadFactory;
4244
import java.util.concurrent.TimeUnit;
4345
import java.util.concurrent.atomic.AtomicLong;
4446

@@ -495,7 +497,8 @@ public final static class Builder {
495497
private boolean optimizingShortfallEvaluator = false;
496498

497499
public Builder() {
498-
executorService = new ScheduledThreadPoolExecutor(1);
500+
ThreadFactory threadFactory = ThreadFactoryBuilder.newBuilder().withNameFormat("fenzo-main").build();
501+
executorService = new ScheduledThreadPoolExecutor(1, threadFactory);
499502
}
500503

501504
/**
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,74 @@
1+
package com.netflix.fenzo.common;
2+
3+
import java.util.Locale;
4+
import java.util.concurrent.ThreadFactory;
5+
import java.util.concurrent.atomic.AtomicLong;
6+
7+
/**
8+
* A ThreadFactory builder based on <a href="https://github.com/google/guava">Guava's</a> ThreadFactoryBuilder.
9+
*/
10+
public final class ThreadFactoryBuilder {
11+
private String nameFormat = null;
12+
private Boolean daemon = null;
13+
14+
private ThreadFactoryBuilder() {
15+
}
16+
17+
/**
18+
* Creates a new {@link ThreadFactoryBuilder} builder.
19+
*/
20+
public static ThreadFactoryBuilder newBuilder() {
21+
return new ThreadFactoryBuilder();
22+
}
23+
24+
/**
25+
* Sets the naming format to use when naming threads ({@link Thread#setName}) which are created
26+
* with this ThreadFactory.
27+
*
28+
* @param nameFormat a {@link String#format(String, Object...)}-compatible format String, to which
29+
* a unique integer (0, 1, etc.) will be supplied as the single parameter. This integer will
30+
* be unique to the built instance of the ThreadFactory and will be assigned sequentially. For
31+
* example, {@code "rpc-pool-%d"} will generate thread names like {@code "rpc-pool-0"},
32+
* {@code "rpc-pool-1"}, {@code "rpc-pool-2"}, etc.
33+
* @return this for the builder pattern
34+
*/
35+
public ThreadFactoryBuilder withNameFormat(String nameFormat) {
36+
this.nameFormat = nameFormat;
37+
return this;
38+
}
39+
40+
/**
41+
* Sets whether or not the created thread will be a daemon thread.
42+
*
43+
* @param daemon whether or not new Threads created with this ThreadFactory will be daemon threads
44+
* @return this for the builder pattern
45+
*/
46+
public ThreadFactoryBuilder withDaemon(boolean daemon) {
47+
this.daemon = daemon;
48+
return this;
49+
}
50+
51+
public ThreadFactory build() {
52+
return build(this);
53+
}
54+
55+
private static ThreadFactory build(ThreadFactoryBuilder builder) {
56+
final String nameFormat = builder.nameFormat;
57+
final Boolean daemon = builder.daemon;
58+
final AtomicLong count = (nameFormat != null) ? new AtomicLong(0) : null;
59+
return runnable -> {
60+
Thread thread = new Thread(runnable);
61+
if (nameFormat != null) {
62+
thread.setName(format(nameFormat, count.getAndIncrement()));
63+
}
64+
if (daemon != null) {
65+
thread.setDaemon(daemon);
66+
}
67+
return thread;
68+
};
69+
}
70+
71+
private static String format(String format, Object... args) {
72+
return String.format(Locale.ROOT, format, args);
73+
}
74+
}

fenzo-core/src/test/java/com/netflix/fenzo/BasicSchedulerTests.java

+25
Original file line numberDiff line numberDiff line change
@@ -552,4 +552,29 @@ public Result evaluate(TaskRequest taskRequest, VirtualMachineCurrentState targe
552552
Assert.assertNotNull(ref.get());
553553
Assert.assertEquals(2, ref.get().size());
554554
}
555+
556+
@Test
557+
public void testTaskBatchSize() {
558+
TaskScheduler taskScheduler = new TaskScheduler.Builder()
559+
.withLeaseOfferExpirySecs(1000000)
560+
.withLeaseRejectAction(virtualMachineLease -> System.out.println("Rejecting offer on host " + virtualMachineLease.hostname()))
561+
.withTaskBatchSizeSupplier(() -> 2L)
562+
.build();
563+
List<VirtualMachineLease> leases = LeaseProvider.getLeases(1, 5, 50, 1, 10);
564+
List<TaskRequest> taskRequests = new ArrayList<>();
565+
taskRequests.add(TaskRequestProvider.getTaskRequest(1, 10, 0));
566+
taskRequests.add(TaskRequestProvider.getTaskRequest(1, 10, 0));
567+
taskRequests.add(TaskRequestProvider.getTaskRequest(1, 10, 0));
568+
taskRequests.add(TaskRequestProvider.getTaskRequest(1, 10, 0));
569+
taskRequests.add(TaskRequestProvider.getTaskRequest(1, 10, 0));
570+
SchedulingResult schedulingResult = taskScheduler.scheduleOnce(taskRequests, leases);
571+
Assert.assertEquals(2, schedulingResult.getResultMap().values().iterator().next().getTasksAssigned().size());
572+
taskRequests = new ArrayList<>();
573+
taskRequests.add(TaskRequestProvider.getTaskRequest(1, 10, 0));
574+
schedulingResult = taskScheduler.scheduleOnce(taskRequests, leases);
575+
Assert.assertEquals(1, schedulingResult.getResultMap().values().iterator().next().getTasksAssigned().size());
576+
taskRequests = new ArrayList<>();
577+
schedulingResult = taskScheduler.scheduleOnce(taskRequests, leases);
578+
Assert.assertEquals(0, schedulingResult.getResultMap().size());
579+
}
555580
}

0 commit comments

Comments
 (0)