-
Notifications
You must be signed in to change notification settings - Fork 3.7k
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
ForkingTaskRunner: Set ActiveProcessorCount for tasks. #12592
Changes from 2 commits
8d4c397
bc519e1
fb84fde
8e63d4b
f9b4e91
1082e5c
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -37,6 +37,7 @@ | |
import com.google.common.io.ByteStreams; | ||
import com.google.common.io.FileWriteMode; | ||
import com.google.common.io.Files; | ||
import com.google.common.math.IntMath; | ||
import com.google.common.util.concurrent.ListenableFuture; | ||
import com.google.common.util.concurrent.ListeningExecutorService; | ||
import com.google.common.util.concurrent.MoreExecutors; | ||
|
@@ -67,6 +68,7 @@ | |
import org.apache.druid.server.metrics.WorkerTaskCountStatsProvider; | ||
import org.apache.druid.tasklogs.TaskLogPusher; | ||
import org.apache.druid.tasklogs.TaskLogStreamer; | ||
import org.apache.druid.utils.JvmUtils; | ||
import org.joda.time.DateTime; | ||
import org.joda.time.Interval; | ||
|
||
|
@@ -75,6 +77,7 @@ | |
import java.io.IOException; | ||
import java.io.InputStream; | ||
import java.io.OutputStream; | ||
import java.math.RoundingMode; | ||
import java.util.ArrayList; | ||
import java.util.Collection; | ||
import java.util.Iterator; | ||
|
@@ -105,6 +108,7 @@ public class ForkingTaskRunner | |
private final StartupLoggingConfig startupLoggingConfig; | ||
private final WorkerConfig workerConfig; | ||
|
||
private volatile int numProcessorsPerTask = -1; | ||
private volatile boolean stopping = false; | ||
|
||
private static final AtomicLong LAST_REPORTED_FAILED_TASK_COUNT = new AtomicLong(); | ||
|
@@ -214,6 +218,13 @@ public TaskStatus call() | |
command.add("-cp"); | ||
command.add(taskClasspath); | ||
|
||
if (numProcessorsPerTask < 1) { | ||
// numProcessorsPerTask is set by start() | ||
throw new ISE("Not started"); | ||
} | ||
|
||
command.add(StringUtils.format("-XX:ActiveProcessorCount=%d", numProcessorsPerTask)); | ||
|
||
Iterables.addAll(command, new QuotableWhiteSpaceSplitter(config.getJavaOpts())); | ||
Iterables.addAll(command, config.getJavaOptsArray()); | ||
|
||
|
@@ -635,7 +646,7 @@ public Optional<ScalingStats> getScalingStats() | |
@Override | ||
public void start() | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Probably needs to be annotated with There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah, I think you're right. I think it makes sense to add the annotations in the impls, because not all impls necessarily need lifecycle management. I added it here. |
||
{ | ||
// No state setup required | ||
setNumProcessorsPerTask(); | ||
} | ||
|
||
@Override | ||
|
@@ -788,6 +799,20 @@ public Long getWorkerSuccessfulTaskCount() | |
return successfulTaskCount - lastReportedSuccessfulTaskCount; | ||
} | ||
|
||
@VisibleForTesting | ||
void setNumProcessorsPerTask() | ||
{ | ||
// Divide number of available processors by the number of tasks. | ||
// This prevents various automatically-sized thread pools from being unreasonably large (we don't want each | ||
// task to size its pools as if it is the only thing on the entire machine). | ||
|
||
final int availableProcessors = JvmUtils.getRuntimeInfo().getAvailableProcessors(); | ||
numProcessorsPerTask = Math.max( | ||
1, | ||
IntMath.divide(availableProcessors, workerConfig.getCapacity(), RoundingMode.CEILING) | ||
); | ||
} | ||
|
||
protected static class ForkingTaskRunnerWorkItem extends TaskRunnerWorkItem | ||
{ | ||
private final Task task; | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -242,6 +242,7 @@ int waitForTaskProcessToComplete(Task task, ProcessHolder processHolder, File lo | |
} | ||
}; | ||
|
||
forkingTaskRunner.setNumProcessorsPerTask(); | ||
final TaskStatus status = forkingTaskRunner.run(NoopTask.create()).get(); | ||
Assert.assertEquals(TaskState.FAILED, status.getStatusCode()); | ||
Assert.assertEquals( | ||
|
@@ -312,6 +313,7 @@ int waitForTaskProcessToComplete(Task task, ProcessHolder processHolder, File lo | |
} | ||
}; | ||
|
||
forkingTaskRunner.setNumProcessorsPerTask(); | ||
final TaskStatus status = forkingTaskRunner.run(task).get(); | ||
Assert.assertEquals(TaskState.SUCCESS, status.getStatusCode()); | ||
Assert.assertNull(status.getErrorMsg()); | ||
|
@@ -373,6 +375,7 @@ int waitForTaskProcessToComplete(Task task, ProcessHolder processHolder, File lo | |
} | ||
}; | ||
|
||
forkingTaskRunner.setNumProcessorsPerTask(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could we call There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I thought about that, but then thought it was best to be as minimal as possible for futureproofing. If we later add more functionality to |
||
final TaskStatus status = forkingTaskRunner.run(task).get(); | ||
Assert.assertEquals(TaskState.FAILED, status.getStatusCode()); | ||
Assert.assertEquals("task failure test", status.getErrorMsg()); | ||
|
@@ -441,6 +444,7 @@ int waitForTaskProcessToComplete(Task task, ProcessHolder processHolder, File lo | |
} | ||
}; | ||
|
||
forkingTaskRunner.setNumProcessorsPerTask(); | ||
forkingTaskRunner.run(task).get(); | ||
Assert.assertTrue(xmxJavaOptsArrayIndex.get() > xmxJavaOptsIndex.get()); | ||
Assert.assertTrue(xmxJavaOptsIndex.get() >= 0); | ||
|
@@ -509,6 +513,7 @@ int waitForTaskProcessToComplete(Task task, ProcessHolder processHolder, File lo | |
} | ||
}; | ||
|
||
forkingTaskRunner.setNumProcessorsPerTask(); | ||
ExecutionException e = Assert.assertThrows(ExecutionException.class, () -> forkingTaskRunner.run(task).get()); | ||
Assert.assertTrue(e.getMessage().endsWith(ForkingTaskRunnerConfig.JAVA_OPTS_ARRAY_PROPERTY | ||
+ " in context of task: " + task.getId() + " must be an array of strings.") | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit: Is this really needed? Won't
TaskRunner.run()
always be called after the lifecycle start method?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It wouldn't be needed if start and run are called in the proper order. It's here as a safety check in case they aren't. (Maybe it would help debug tests that are doing things incorrectly.)