-
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
Conversation
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).
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.
Added minor comments.
@@ -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 comment
The reason will be displayed to describe this comment to others. Learn more.
Probably needs to be annotated with @LifecycleStart
. I wonder if just annotating the interface method TaskRunner.start()
would work so that the implementations don't need to worry about this.
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.
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.
@@ -214,6 +218,13 @@ public TaskStatus call() | |||
command.add("-cp"); | |||
command.add(taskClasspath); | |||
|
|||
if (numProcessorsPerTask < 1) { |
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.)
@@ -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 comment
The reason will be displayed to describe this comment to others. Learn more.
Could we call forkingTaskRunner.start()
here instead to avoid exposing the setNumProcessorsPerTask()
as a @VisibleForTesting
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.
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 start()
, we wouldn't necessarily want that to execute in these tests.
@gianm - what will be the effect of this change on existing ingestion jobs? The thread pool sizes are going to change after this change on the same hardware. Users could see changes in performance for real-time nodes if they have not configured the size of the processing thread pool explicitly. |
Yeah essentially any autoconfigured thread pool will act as if they have a slice of the machine instead of the entire machine. There are a lot of these. I think for most of them, users won't notice, and this change will be an improvement due to fewer overall threads on the machine. Users may notice the processing pool changing size if they weren't explicitly setting it. That's worth calling out in the release notes, so it's good that you added a release notes label. Thanks. |
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.
LGTM!
Many integration tests are failing. There might be something wrong with the way the commandline is being constructed. I'll look into it when I get a chance. |
The issue was that ForkingTaskRunner needs to have its lifecycle managed. The latest patch fixes it. |
Could you elaborate on the specific processing thread pools affected by this setting around ingest? I'm trying to work out how to revert this setting (or at least revert the behaviour) to see if this is the underlying cause in our ingest speed issues post upgrade to 24.0.0 |
|
what kind of speed issues are you observing? |
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).
On large machines, this solves a common cause of OutOfMemoryError due to
"unable to create native thread".