Skip to content

Conversation

@wjones127
Copy link
Member

@wjones127 wjones127 commented Feb 4, 2022

Test failure on my branch: https://github.com/wjones127/arrow/runs/5068285944?check_suite_focus=true#step:18:27831

This only fails on Windows. On any platform, when USE_THREADS = FALSE we cannot execute an exec plan that joins after scanning from a dataset.

The source node executes the InputRecieved() callback in the IO thread, while the HashJoin wasn't anticipating another thread if executor is null (which USE_THREADS=FALSE sets). This PR adds one to the thread local state vector size to anticipate this case of 1 main thread + 1 IO thread.

@github-actions
Copy link

github-actions bot commented Feb 4, 2022

@github-actions
Copy link

github-actions bot commented Feb 4, 2022

⚠️ Ticket has not been started in JIRA, please click 'Start Progress'.

@wjones127
Copy link
Member Author

My best guess right now is that some memory is getting clobbered by another thread. Is there a way to execute on a single thread somehow? Tried set_cpu_count(1) and set_io_thread_count(1) (and also USE_THREADS=FALSE is default on Windows), but same issue.

@jonkeane Any thoughts on the threading settings?
@bkietz I know isn't a lot to go off of, but anything stand out to you in the above tracebacks?

@wjones127 wjones127 force-pushed the ARROW-14908-windows-join-crash branch from 3de21c0 to 507050b Compare February 9, 2022 22:21
@wjones127 wjones127 changed the title ARROW-14908: [C++][R] Dataset hash join segfaults on Windows [WIP] ARROW-14908: [C++][R] Dataset hash join segfaults on Windows Feb 11, 2022
@wjones127 wjones127 marked this pull request as ready for review February 11, 2022 17:39
The main thread touches the thread_indexer before it's used by the
worker thread, so we need to make sure it has capacity for both
threads.
@wjones127 wjones127 force-pushed the ARROW-14908-windows-join-crash branch from 9a510cb to a61b7a4 Compare February 11, 2022 17:40
Copy link
Member

@jonkeane jonkeane left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for all of your work on this, I know it's been a lot. A few comments (mostly on linting)

wjones127 and others added 2 commits February 11, 2022 10:50
Co-authored-by: Jonathan Keane <jkeane@gmail.com>
@wjones127 wjones127 requested a review from westonpace February 11, 2022 20:09
Copy link
Member

@westonpace westonpace left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks good! Thanks for figuring this out.

Co-authored-by: Weston Pace <weston.pace@gmail.com>
@ursabot
Copy link

ursabot commented Feb 14, 2022

Benchmark runs are scheduled for baseline = 5ad5ddc and contender = 7b5efe4. 7b5efe4 is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
Conbench compare runs links:
[Finished ⬇️0.0% ⬆️0.0%] ec2-t3-xlarge-us-east-2
[Finished ⬇️0.81% ⬆️0.0%] test-mac-arm
[Failed ⬇️0.36% ⬆️0.0%] ursa-i9-9960x
[Finished ⬇️0.13% ⬆️0.09%] ursa-thinkcentre-m75q
Supported benchmarks:
ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python. Runs only benchmarks with cloud = True
test-mac-arm: Supported benchmark langs: C++, Python, R
ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java

kszucs pushed a commit to kszucs/arrow that referenced this pull request Mar 3, 2022
Test failure on my branch: https://github.com/wjones127/arrow/runs/5068285944?check_suite_focus=true#step:18:27831

~This only fails on Windows~. On any platform, when `USE_THREADS = FALSE` we cannot execute an exec plan that joins after scanning from a dataset.

The source node executes the `InputRecieved()` callback in the IO thread, while the HashJoin wasn't anticipating another thread if executor is null (which `USE_THREADS=FALSE` sets). This PR adds one to the thread local state vector size to anticipate this case of 1 main thread + 1 IO thread.

Closes apache#12339 from wjones127/ARROW-14908-windows-join-crash

Authored-by: Will Jones <willjones127@gmail.com>
Signed-off-by: Weston Pace <weston.pace@gmail.com>
westonpace pushed a commit that referenced this pull request Apr 21, 2022
In #12339 we added one, which enabled joining one table to one dataset using `use_threads=false`. However, I found that joining two datasets hit the thread limit.

There are plans to find [a long-term fix](https://issues.apache.org/jira/browse/ARROW-16072) that can run these operations synchronously with fewer threads, but that won't be ready for the next release.

As a temporary fix for 8.0.0, I propose just bumping up the `local_states_` capacity.

Closes #12845 from wjones127/ARROW-15718-multiple-datasets

Authored-by: Will Jones <willjones127@gmail.com>
Signed-off-by: Weston Pace <weston.pace@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants