Skip to content

GH-46454: [C++][Dataset][Acero] Preserve order when writting with TeeNode #46455

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

Open
wants to merge 1 commit into
base: main
Choose a base branch
from
Open
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
23 changes: 21 additions & 2 deletions cpp/src/arrow/dataset/file_base.cc
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

#include "arrow/dataset/file_base.h"

#include "arrow/acero/accumulation_queue.h"
#include "arrow/acero/exec_plan.h"

#include <algorithm>
Expand Down Expand Up @@ -559,13 +560,18 @@ Result<acero::ExecNode*> MakeWriteNode(acero::ExecPlan* plan,

namespace {

class TeeNode : public acero::MapNode {
class TeeNode : public acero::MapNode,
public arrow::acero::util::SerialSequencingQueue::Processor {
public:
TeeNode(acero::ExecPlan* plan, std::vector<acero::ExecNode*> inputs,
std::shared_ptr<Schema> output_schema,
FileSystemDatasetWriteOptions write_options)
: MapNode(plan, std::move(inputs), std::move(output_schema)),
write_options_(std::move(write_options)) {}
write_options_(std::move(write_options)) {
if (write_options.preserve_order) {
sequencer_ = acero::util::SerialSequencingQueue::Make(this);
}
}

Status StartProducing() override {
ARROW_ASSIGN_OR_RAISE(
Expand All @@ -592,6 +598,18 @@ class TeeNode : public acero::MapNode {

const char* kind_name() const override { return "TeeNode"; }

Status InputReceived(ExecNode* input, ExecBatch batch) override {
DCHECK_EQ(input, inputs_[0]);
if (sequencer_) {
return sequencer_->InsertBatch(std::move(batch));
}
return Process(std::move(batch));
}

Status Process(ExecBatch batch) override {
return acero::MapNode::InputReceived(inputs_[0], batch);
}

void Finish() override { dataset_writer_->Finish(); }

Result<compute::ExecBatch> ProcessBatch(compute::ExecBatch batch) override {
Expand Down Expand Up @@ -625,6 +643,7 @@ class TeeNode : public acero::MapNode {
std::unique_ptr<internal::DatasetWriter> dataset_writer_;
FileSystemDatasetWriteOptions write_options_;
std::atomic<int32_t> backpressure_counter_ = 0;
std::unique_ptr<acero::util::SerialSequencingQueue> sequencer_{nullptr};
};

} // namespace
Expand Down
Loading