[
https://issues.apache.org/jira/browse/ARROW-10117?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17347712#comment-17347712
]
Weston Pace commented on ARROW-10117:
-------------------------------------
I'm willing/interested in taking a stab at this but I'm wondering a bit about
evaluation and warning up-front that I don't expect to see any benefit
anywhere. Pretty much all existing parallelism can be broken down into two
categories:
Task-per-batch - In this case we are creating a new task to handle a new file
or block of data. For example, when scanning a dataset with multiple files we
create a task for each file. When doing asynchronous readahead we create a
task per record batch.
Task-per-column - In a few places we take this approach to divide expensive
work across columns.
For future work (execution engine) the latest discussion seems to be around
"morsel-driven execution model". If I am stripping away the academia properly
a "morsel-driven execution model" simply means "task-per-batch".
Task-per-batch execution will never really benefit from a work-stealing
scheduler. Each task is operating on a brand new chunk of data. There isn't
any cache coherency to benefit from.
Task-per-column might benefit in two situations:
* If there is enough other work going on at the same time then multiple
queues would be a way to disable task-per-column
* If used within a task-per-batch parallel pipeline this would disable
task-per-column dynamically if there were sufficient batches (this is basically
a special case of the above bullet).
There is one other special case of parallelism and that is the background
readers. One could theorize that multiple queues synchronized across the I/O
and CPU thread pools might allow you to match up your I/O core with your
processing core. However, in practice I have found that if the pipeline is not
I/O bound then it's a moot point and if the pipeline is I/O bound then the
context switch overhead is minimal.
TL;DR: I will create an artificial benchmark that should see obvious benefits
from work stealing. I'll create a work stealing thread pool using the
artificial benchmark to verify I did things correctly. I'll then turn it on,
run conbench, and cross my fingers but I'm not expecting we will see any change
(but I'm more than happy to be pleasantly surprised).
> [C++] Implement work-stealing scheduler / multiple queues in ThreadPool
> -----------------------------------------------------------------------
>
> Key: ARROW-10117
> URL: https://issues.apache.org/jira/browse/ARROW-10117
> Project: Apache Arrow
> Issue Type: Improvement
> Components: C++
> Reporter: Wes McKinney
> Priority: Major
>
> This involves a change from a single task queue shared amongst all threads to
> a per-thread task queue and the ability for idle threads to take tasks from
> other threads' queues (work stealing).
> As part of this, the task submission API would need to be evolved in some
> fashion to allow for tasks related to a particular workload to end up in the
> same task queue
--
This message was sent by Atlassian Jira
(v8.3.4#803005)