Skip to content

Commit 11481f4

Browse files
author
Rafał Hibner
committed
Merge branch 'arrowPipeNoQueue' into combined3
2 parents 9f0970d + 6f1e1e3 commit 11481f4

File tree

2 files changed

+23
-59
lines changed

2 files changed

+23
-59
lines changed

cpp/src/arrow/acero/pipe_node.cc

Lines changed: 18 additions & 52 deletions
Original file line numberDiff line numberDiff line change
@@ -237,11 +237,24 @@ PipeSource::PipeSource() {}
237237
Status PipeSource::Initialize(Pipe* pipe) {
238238
if (pipe_) return Status::Invalid("Pipe:" + pipe->PipeName() + " has multiple sinks");
239239
pipe_ = pipe;
240+
backpressure_source_.AddController(pipe);
240241
return Status::OK();
241242
}
242243

243-
void PipeSource::Pause(int32_t counter) { pipe_->Pause(this, counter); }
244-
void PipeSource::Resume(int32_t counter) { pipe_->Resume(this, counter); }
244+
void PipeSource::Pause(int32_t counter) {
245+
auto lock = mutex_.Lock();
246+
if (backpressure_counter < counter) {
247+
backpressure_counter = counter;
248+
backpressure_source_.Pause();
249+
}
250+
}
251+
void PipeSource::Resume(int32_t counter) {
252+
auto lock = mutex_.Lock();
253+
if (backpressure_counter < counter) {
254+
backpressure_counter = counter;
255+
backpressure_source_.Resume();
256+
}
257+
}
245258
Status PipeSource::StopProducing() {
246259
if (pipe_) return pipe_->StopProducing(this);
247260
// stopped before initialization
@@ -265,67 +278,20 @@ Pipe::Pipe(ExecPlan* plan, std::string pipe_name,
265278
std::unique_ptr<BackpressureControl> ctrl,
266279
std::function<Status()> stopProducing, Ordering ordering, bool pause_on_any,
267280
bool stop_on_any)
268-
: plan_(plan),
281+
: BackpressureCombiner(std::move(ctrl), pause_on_any),
282+
plan_(plan),
269283
ordering_(ordering),
270284
pipe_name_(pipe_name),
271-
ctrl_(std::move(ctrl)),
272285
stopProducing_(stopProducing),
273-
pause_on_any_(pause_on_any),
274286
stop_on_any_(stop_on_any) {}
275287

276288
const Ordering& Pipe::ordering() const { return ordering_; }
277289

278-
void Pipe::Pause(PipeSource* output, int counter) {
279-
auto lock = mutex_.Lock();
280-
auto& state = state_[output];
281-
if (state.backpressure_counter < counter) {
282-
state.backpressure_counter = counter;
283-
if (!state.paused && !state.stopped) {
284-
state.paused = true;
285-
size_t paused_count = ++paused_count_;
286-
if (pause_on_any_) {
287-
if (paused_count == 1) {
288-
ctrl_->Pause();
289-
}
290-
} else {
291-
if (paused_count == CountSources() - stopped_count_) {
292-
ctrl_->Pause();
293-
}
294-
}
295-
}
296-
}
297-
}
298-
299-
void Pipe::Resume(PipeSource* output, int counter) {
300-
auto lock = mutex_.Lock();
301-
auto& state = state_[output];
302-
if (state.backpressure_counter < counter) {
303-
state.backpressure_counter = counter;
304-
DoResume(state);
305-
}
306-
}
307-
308-
void Pipe::DoResume(SourceState& state) {
309-
if (state.paused && !state.stopped) {
310-
state.paused = false;
311-
size_t paused_count = --paused_count_;
312-
if (pause_on_any_) {
313-
if (paused_count == 0) {
314-
ctrl_->Resume();
315-
}
316-
} else {
317-
if (paused_count == CountSources() - stopped_count_ - 1) {
318-
ctrl_->Resume();
319-
}
320-
}
321-
}
322-
}
323-
324290
Status Pipe::StopProducing(PipeSource* output) {
325291
auto lock = mutex_.Lock();
326292
auto& state = state_[output];
327293
DCHECK(!state.stopped);
328-
DoResume(state);
294+
BackpressureCombiner::Stop();
329295
state.stopped = true;
330296
size_t stopped_count = ++stopped_count_;
331297
if (stop_on_any_) {

cpp/src/arrow/acero/pipe_node.h

Lines changed: 5 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717

1818
#pragma once
1919
#include <string>
20+
#include "arrow/acero/backpressure.h"
2021
#include "arrow/acero/exec_plan.h"
2122
#include "arrow/acero/options.h"
2223
#include "arrow/acero/visibility.h"
@@ -47,12 +48,15 @@ class PipeSource {
4748
virtual Status HandleInputFinished(int total_batches) = 0;
4849

4950
Pipe* pipe_{nullptr};
51+
BackpressureCombiner::Source backpressure_source_;
52+
Mutex mutex_;
53+
int backpressure_counter{0};
5054
};
5155

5256
/// @brief Provides pipe like infastructure for Acero. It isa center element for
5357
/// pipe_sink/pipe_tee and pipe_source infrastructure. Can also be used to create
5458
/// auxiliarty outputs(pipes) for ExecNodes.
55-
class ARROW_ACERO_EXPORT Pipe {
59+
class ARROW_ACERO_EXPORT Pipe : public BackpressureCombiner {
5660
public:
5761
Pipe(ExecPlan* plan, std::string pipe_name, std::unique_ptr<BackpressureControl> ctrl,
5862
std::function<Status()> stopProducing, Ordering ordering = Ordering::Unordered(),
@@ -96,16 +100,13 @@ class ARROW_ACERO_EXPORT Pipe {
96100
friend class PipeSource;
97101

98102
struct SourceState {
99-
bool paused{false};
100103
bool stopped{false};
101-
int backpressure_counter{0};
102104
};
103105

104106
// Backpresurre interface for PipeSource
105107
void Pause(PipeSource* output, int counter);
106108
// Backpresurre interface for PipeSource
107109
void Resume(PipeSource* output, int counter);
108-
void DoResume(SourceState& state);
109110
//
110111
Status StopProducing(PipeSource* output);
111112

@@ -118,13 +119,10 @@ class ARROW_ACERO_EXPORT Pipe {
118119
// backpressure
119120
std::unordered_map<PipeSource*, SourceState> state_;
120121
Mutex mutex_;
121-
std::atomic_size_t paused_count_{0};
122-
std::unique_ptr<BackpressureControl> ctrl_;
123122
// stopProducing
124123
std::atomic_size_t stopped_count_{0};
125124
std::function<Status()> stopProducing_;
126125

127-
const bool pause_on_any_;
128126
const bool stop_on_any_;
129127
};
130128

0 commit comments

Comments
 (0)