This is an automated email from the ASF dual-hosted git repository.
apitrou pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/arrow.git
The following commit(s) were added to refs/heads/main by this push:
new 9a7662b41b GH-39582: [C++][Acero] Increase size of Acero TempStack
(#40007)
9a7662b41b is described below
commit 9a7662b41b77a40a76b07435b3eff4fec7454596
Author: Sten Larsson <[email protected]>
AuthorDate: Mon Feb 26 18:02:44 2024 +0100
GH-39582: [C++][Acero] Increase size of Acero TempStack (#40007)
We have had problems for a long time with a specific batch job that
combines data from different sources. There is something in the data causing an
Acero execution plan to hang or crash at random. The problem has been
reproduced since Arrow 11.0.0, originally in Ruby, but it has also in Python.
There is unfortunately no test case that reliably reproduces the issue in a
release build.
However, in a debug build we can see that the batch job causes an overflow
on the temp stack in arrow/cpp/src/arrow/compute/util.cc:38. Increasing the
size of the stack created in the Acero QueryContext works around the issue, but
a real fix should be investigated separately.
**This PR contains a "Critical Fix".**
* Closes: #39582
Lead-authored-by: Sten Larsson <[email protected]>
Co-authored-by: Antoine Pitrou <[email protected]>
Signed-off-by: Antoine Pitrou <[email protected]>
---
cpp/src/arrow/acero/query_context.cc | 2 +-
cpp/src/arrow/compute/util.cc | 15 ++++++++-------
2 files changed, 9 insertions(+), 8 deletions(-)
diff --git a/cpp/src/arrow/acero/query_context.cc
b/cpp/src/arrow/acero/query_context.cc
index 9f838508fc..a27397d120 100644
--- a/cpp/src/arrow/acero/query_context.cc
+++ b/cpp/src/arrow/acero/query_context.cc
@@ -53,7 +53,7 @@ size_t QueryContext::max_concurrency() const { return
thread_indexer_.Capacity()
Result<util::TempVectorStack*> QueryContext::GetTempStack(size_t thread_index)
{
if (!tld_[thread_index].is_init) {
RETURN_NOT_OK(tld_[thread_index].stack.Init(
- memory_pool(), 8 * util::MiniBatch::kMiniBatchLength *
sizeof(uint64_t)));
+ memory_pool(), 32 * util::MiniBatch::kMiniBatchLength *
sizeof(uint64_t)));
tld_[thread_index].is_init = true;
}
return &tld_[thread_index].stack;
diff --git a/cpp/src/arrow/compute/util.cc b/cpp/src/arrow/compute/util.cc
index c55143af0c..2058ba9f30 100644
--- a/cpp/src/arrow/compute/util.cc
+++ b/cpp/src/arrow/compute/util.cc
@@ -32,17 +32,18 @@ using internal::CpuInfo;
namespace util {
void TempVectorStack::alloc(uint32_t num_bytes, uint8_t** data, int* id) {
- int64_t old_top = top_;
- top_ += PaddedAllocationSize(num_bytes) + 2 * sizeof(uint64_t);
- // Stack overflow check
- ARROW_DCHECK(top_ <= buffer_size_);
- *data = buffer_->mutable_data() + old_top + sizeof(uint64_t);
+ int64_t new_top = top_ + PaddedAllocationSize(num_bytes) + 2 *
sizeof(uint64_t);
+ // Stack overflow check (see GH-39582).
+ // XXX cannot return a regular Status because most consumers do not either.
+ ARROW_CHECK_LE(new_top, buffer_size_) << "TempVectorStack::alloc overflow";
+ *data = buffer_->mutable_data() + top_ + sizeof(uint64_t);
// We set 8 bytes before the beginning of the allocated range and
// 8 bytes after the end to check for stack overflow (which would
// result in those known bytes being corrupted).
- reinterpret_cast<uint64_t*>(buffer_->mutable_data() + old_top)[0] = kGuard1;
- reinterpret_cast<uint64_t*>(buffer_->mutable_data() + top_)[-1] = kGuard2;
+ reinterpret_cast<uint64_t*>(buffer_->mutable_data() + top_)[0] = kGuard1;
+ reinterpret_cast<uint64_t*>(buffer_->mutable_data() + new_top)[-1] = kGuard2;
*id = num_vectors_++;
+ top_ = new_top;
}
void TempVectorStack::release(int id, uint32_t num_bytes) {