-
Notifications
You must be signed in to change notification settings - Fork 1.3k
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
join statement causes panic #7931
Comments
please assign to me |
Thank you @guojidan and thank you @yukkit for a nice reproducer. FWIW we have seen this error when trying to create String arrays with "very large" strings Basically since a So if somehow your query output tries to make a column with more than 2GB of string data it will panic In this case the I ran this reproducer locally and got a stack trace $ RUST_BACKTRACE=1 datafusion-cli -c 'SELECT * FROM air INNER JOIN sea ON sea.station = air.station;'
DataFusion CLI v32.0.0
thread 'tokio-runtime-worker' panicked at /Users/alamb/.cargo/registry/src/index.crates.io-6f17d22bba15001f/arrow-select-48.0.0/src/take.rs:424:41:
offset overflow
stack backtrace:
0: rust_begin_unwind
at /rustc/cc66ad468955717ab92600c770da8c1601a4ff33/library/std/src/panicking.rs:595:5
1: core::panicking::panic_fmt
at /rustc/cc66ad468955717ab92600c770da8c1601a4ff33/library/core/src/panicking.rs:67:14
2: core::panicking::panic_display
at /rustc/cc66ad468955717ab92600c770da8c1601a4ff33/library/core/src/panicking.rs:150:5
3: core::panicking::panic_str
at /rustc/cc66ad468955717ab92600c770da8c1601a4ff33/library/core/src/panicking.rs:134:5
4: core::option::expect_failed
at /rustc/cc66ad468955717ab92600c770da8c1601a4ff33/library/core/src/option.rs:1988:5
5: core::option::Option<T>::expect
at /rustc/cc66ad468955717ab92600c770da8c1601a4ff33/library/core/src/option.rs:898:21
6: arrow_select::take::take_bytes
at /Users/alamb/.cargo/registry/src/index.crates.io-6f17d22bba15001f/arrow-select-48.0.0/src/take.rs:424:41
7: arrow_select::take::take_impl
at /Users/alamb/.cargo/registry/src/index.crates.io-6f17d22bba15001f/arrow-select-48.0.0/src/take.rs:142:25
8: arrow_select::take::take
at /Users/alamb/.cargo/registry/src/index.crates.io-6f17d22bba15001f/arrow-select-48.0.0/src/take.rs:92:5
9: datafusion_physical_plan::joins::hash_join::equal_rows_arr
at /Users/alamb/Software/arrow-datafusion/datafusion/physical-plan/src/joins/hash_join.rs:876:20
10: datafusion_physical_plan::joins::hash_join::build_equal_condition_join_indices
at /Users/alamb/Software/arrow-datafusion/datafusion/physical-plan/src/joins/hash_join.rs:840:5
11: datafusion_physical_plan::joins::hash_join::HashJoinStream::poll_next_impl::{{closure}}
at /Users/alamb/Software/arrow-datafusion/datafusion/physical-plan/src/joins/hash_join.rs:955:46
12: core::task::poll::Poll<T>::map
at /rustc/cc66ad468955717ab92600c770da8c1601a4ff33/library/core/src/task/poll.rs:53:43
13: datafusion_physical_plan::joins::hash_join::HashJoinStream::poll_next_impl
at /Users/alamb/Software/arrow-datafusion/datafusion/physical-plan/src/joins/hash_join.rs:947:14
14: <datafusion_physical_plan::joins::hash_join::HashJoinStream as futures_core::stream::Stream>::poll_next
at /Users/alamb/Software/arrow-datafusion/datafusion/physical-plan/src/joins/hash_join.rs:1056:9
15: <core::pin::Pin<P> as futures_core::stream::Stream>::poll_next
at /Users/alamb/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-core-0.3.28/src/stream.rs:120:9
16: futures_util::stream::stream::StreamExt::poll_next_unpin
at /Users/alamb/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-util-0.3.28/src/stream/stream/mod.rs:1632:9
17: datafusion_physical_plan::coalesce_batches::CoalesceBatchesStream::poll_next_inner
at /Users/alamb/Software/arrow-datafusion/datafusion/physical-plan/src/coalesce_batches.rs:226:42
18: <datafusion_physical_plan::coalesce_batches::CoalesceBatchesStream as futures_core::stream::Stream>::poll_next
at /Users/alamb/Software/arrow-datafusion/datafusion/physical-plan/src/coalesce_batches.rs:204:20
19: <core::pin::Pin<P> as futures_core::stream::Stream>::poll_next
at /Users/alamb/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-core-0.3.28/src/stream.rs:120:9
20: futures_util::stream::stream::StreamExt::poll_next_unpin
at /Users/alamb/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-util-0.3.28/src/stream/stream/mod.rs:1632:9
21: <futures_util::stream::stream::next::Next<St> as core::future::future::Future>::poll
at /Users/alamb/.cargo/registry/src/index.crates.io-6f17d22bba15001f/futures-util-0.3.28/src/stream/stream/next.rs:32:21
22: datafusion_physical_plan::stream::RecordBatchReceiverStreamBuilder::run_input::{{closure}}
at /Users/alamb/Software/arrow-datafusion/datafusion/physical-plan/src/stream.rs:242:50
23: tokio::runtime::task::core::Core<T,S>::poll::{{closure}}
at /Users/alamb/.cargo/registry/src/index.crates.io-6f17d22bba15001f/tokio-1.33.0/src/runtime/task/core.rs:328:17
24: tokio::loom::std::unsafe_cell::UnsafeCell<T>::with_mut
at /Users/alamb/.cargo/registry/src/index.crates.io-6f17d22bba15001f/tokio-1.33.0/src/loom/std/unsafe_cell.rs:16:9
25: tokio::runtime::task::core::Core<T,S>::poll
at /Users/alamb/.cargo/registry/src/index.crates.io-6f17d22bba15001f/tokio-1.33.0/src/runtime/task/core.rs:317:13
26: tokio::runtime::task::harness::poll_future::{{closure}}
at /Users/alamb/.cargo/registry/src/index.crates.io-6f17d22bba15001f/tokio-1.33.0/src/runtime/task/harness.rs:485:19
27: <core::panic::unwind_safe::AssertUnwindSafe<F> as core::ops::function::FnOnce<()>>::call_once
at /rustc/cc66ad468955717ab92600c770da8c1601a4ff33/library/core/src/panic/unwind_safe.rs:271:9 is trying to make a single giant array batch with a single call to |
I suppose this issue may be closed -- the reproducer based on data files from the description Reproducer code
is able to complete with current main version(840499f), while running it with 35.0.0 / 34.0.0 ends up with OOM |
Nice work @korowa (again) |
I can conform it seems to work fine, I had another scenario with the same issue and it works against the main branch. |
Describe the bug
offset overflow
may occur when I perform an inner joinTo Reproduce
The reason is that there is an overflow during type conversion.
https://github.com/apache/arrow-rs/blob/48.0.0/arrow-select/src/take.rs#L424
Expected behavior
No response
Additional context
git commit: 4881b5d
data.zip
I don't have time to deal with this problem at the moment, can anyone help solve it
The text was updated successfully, but these errors were encountered: