Skip to content

Commit 78fb2ed

Browse files
committed
ARROW-15410: [C++][Datasets] Improve memory usage of datasets API when scanning parquet
This PR changes a few things. * The default file readahead is changed to 4. This doesn't seem to affect performance on HDD/SSD and users should already be doing special tuning for S3. Besides, in many cases, users are reading IPC/Parquet files that have many row groups and so we already have sufficient I/O parallelism. This is important for bringing down the overall memory usage as can be seen in the formula below. * The default batch readahead is changed to 16. Previously, when we were doing filtering and projection within the scanner, it made sense to read many batches ahead (generally want at least 2 * # of CPUs in that case). Now that the exec plan is doing the computation the exec plan buffering is instead handled by kDefaultBackpressureLowBytes and kDefaultBackpressureHighBytes. * Moves around the parquet readahead a bit. The previous version would read ahead N row groups. Now we always read ahead exactly 1 row group but we read ahead N batches (this may mean that we read ahead more than 1 row group if the batch size is much larger than the row group size). * Backpressure now utilizes the pause/resume producing signals in the execution plan. I've adding a `counter` argument to the calls to help deal with the challenges that arise when we try and sequence backpressure signals. Partly this was to add support for monitoring backpressure (for tests). Partly it is because I have since become more aware of the reasons for these signals. They are needed to allow for backpressure from the aggregate & join nodes. * Sink backpressure can now be monitored. This makes it easier to test and could be potentially useful to a user that wanted to know when they are consuming the plan too slowly. * Changes the default scanner batch size to 128Ki rows. Now that we have more or less decoupled the scanning batch size from the row group size we can pass smaller batches through the scanner. This makes it easier to get parallelism on small datasets.. Putting this altogether the scanner should now buffer in memory: MAX(fragment_readahead * row_group_size_bytes * 2, fragment_readahead * batch_readahead * batch_size_bytes) The exec plan sink node should buffer ~ kDefaultBackpressureHighBytes bytes. The exec plan itself can have some number of tasks in flight but, assuming there are no pipeline breakers, this will be limited to the number of threads in the CPU thread pool and so it should be parallelism * batch_size_bytes. Adding those together should give the total RAM usage of a plan being read via a sink node that doesn't have any pipeline breakers. When the sink is a write node then there is a separate backpressure consideration based on # of rows (we can someday change this to be # of bytes but it would be a bit tricky at the moment because we need to balance this with the other write parameters like min_rows_per_group). So, given the parquet dataset mentioned in the JIRA (21 files, 10 million rows each, 10 row groups each) and knowing that 1 row group is ~140MB when decompressed into Arrow format we should get the following default memory usage: Scanner readahead = MAX(4 * 140MB * 2, 4 * 16 * 17.5MB) = MAX(1120MB, 1120MB) = 1120MB Sink readahead ~ 1GiB Total RAM usage should then be ~2GiB. - [x] Add tests to verify memory usage - [ ] ~~Update docs to mention that S3 users may want to increase the fragment readahead but this will come at the cost of more RAM usage.~~ - [ ] ~~Update docs to give some of this "expected memory usage" information~~ Closes apache#12228 from westonpace/feature/ARROW-15410--improve-dataset-parquet-memory-usage Authored-by: Weston Pace <weston.pace@gmail.com> Signed-off-by: Weston Pace <weston.pace@gmail.com>
1 parent 6c1a160 commit 78fb2ed

28 files changed

Lines changed: 593 additions & 477 deletions

cpp/examples/arrow/compute_register_example.cc

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -92,8 +92,12 @@ class ExampleNode : public cp::ExecNode {
9292
return arrow::Status::OK();
9393
}
9494

95-
void ResumeProducing(ExecNode* output) override {}
96-
void PauseProducing(ExecNode* output) override {}
95+
void ResumeProducing(ExecNode* output, int32_t counter) override {
96+
inputs_[0]->ResumeProducing(this, counter);
97+
}
98+
void PauseProducing(ExecNode* output, int32_t counter) override {
99+
inputs_[0]->PauseProducing(this, counter);
100+
}
97101

98102
void StopProducing(ExecNode* output) override { inputs_[0]->StopProducing(this); }
99103
void StopProducing() override { inputs_[0]->StopProducing(); }

cpp/examples/arrow/engine_substrait_consumption.cc

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,8 @@ class IgnoringConsumer : public cp::SinkNodeConsumer {
4040
public:
4141
explicit IgnoringConsumer(size_t tag) : tag_{tag} {}
4242

43-
arrow::Status Init(const std::shared_ptr<arrow::Schema>& schema) override {
43+
arrow::Status Init(const std::shared_ptr<arrow::Schema>& schema,
44+
cp::BackpressureControl* backpressure_control) override {
4445
return arrow::Status::OK();
4546
}
4647

cpp/examples/arrow/execution_plan_documentation_examples.cc

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -591,7 +591,8 @@ arrow::Status SourceConsumingSinkExample(cp::ExecContext& exec_context) {
591591
CustomSinkNodeConsumer(std::atomic<uint32_t>* batches_seen, arrow::Future<> finish)
592592
: batches_seen(batches_seen), finish(std::move(finish)) {}
593593

594-
arrow::Status Init(const std::shared_ptr<arrow::Schema>& schema) override {
594+
arrow::Status Init(const std::shared_ptr<arrow::Schema>& schema,
595+
cp::BackpressureControl* backpressure_control) override {
595596
return arrow::Status::OK();
596597
}
597598

cpp/src/arrow/compute/exec/aggregate_node.cc

Lines changed: 14 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -232,9 +232,13 @@ class ScalarAggregateNode : public ExecNode {
232232
return Status::OK();
233233
}
234234

235-
void PauseProducing(ExecNode* output) override { EVENT(span_, "PauseProducing"); }
235+
void PauseProducing(ExecNode* output, int32_t counter) override {
236+
inputs_[0]->PauseProducing(this, counter);
237+
}
236238

237-
void ResumeProducing(ExecNode* output) override { EVENT(span_, "ResumeProducing"); }
239+
void ResumeProducing(ExecNode* output, int32_t counter) override {
240+
inputs_[0]->ResumeProducing(this, counter);
241+
}
238242

239243
void StopProducing(ExecNode* output) override {
240244
DCHECK_EQ(output, outputs_[0]);
@@ -598,9 +602,15 @@ class GroupByNode : public ExecNode {
598602
return Status::OK();
599603
}
600604

601-
void PauseProducing(ExecNode* output) override { EVENT(span_, "PauseProducing"); }
605+
void PauseProducing(ExecNode* output, int32_t counter) override {
606+
// TODO(ARROW-16260)
607+
// Without spillover there is way to handle backpressure in this node
608+
}
602609

603-
void ResumeProducing(ExecNode* output) override { EVENT(span_, "ResumeProducing"); }
610+
void ResumeProducing(ExecNode* output, int32_t counter) override {
611+
// TODO(ARROW-16260)
612+
// Without spillover there is way to handle backpressure in this node
613+
}
604614

605615
void StopProducing(ExecNode* output) override {
606616
EVENT(span_, "StopProducing");

cpp/src/arrow/compute/exec/exec_plan.cc

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -395,9 +395,13 @@ Status MapNode::StartProducing() {
395395
return Status::OK();
396396
}
397397

398-
void MapNode::PauseProducing(ExecNode* output) { EVENT(span_, "PauseProducing"); }
398+
void MapNode::PauseProducing(ExecNode* output, int32_t counter) {
399+
inputs_[0]->PauseProducing(this, counter);
400+
}
399401

400-
void MapNode::ResumeProducing(ExecNode* output) { EVENT(span_, "ResumeProducing"); }
402+
void MapNode::ResumeProducing(ExecNode* output, int32_t counter) {
403+
inputs_[0]->ResumeProducing(this, counter);
404+
}
401405

402406
void MapNode::StopProducing(ExecNode* output) {
403407
DCHECK_EQ(output, outputs_[0]);

cpp/src/arrow/compute/exec/exec_plan.h

Lines changed: 28 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -194,6 +194,24 @@ class ARROW_EXPORT ExecNode {
194194
// - A method allows passing a ProductionHint asynchronously from an output node
195195
// (replacing PauseProducing(), ResumeProducing(), StopProducing())
196196

197+
// Concurrent calls to PauseProducing and ResumeProducing can be hard to sequence
198+
// as they may travel at different speeds through the plan.
199+
//
200+
// For example, consider a resume that comes quickly after a pause. If the source
201+
// receives the resume before the pause the source may think the destination is full
202+
// and halt production which would lead to deadlock.
203+
//
204+
// To resolve this a counter is sent for all calls to pause/resume. Only the call with
205+
// the highest counter value is valid. So if a call to PauseProducing(5) comes after
206+
// a call to ResumeProducing(6) then the source should continue producing.
207+
//
208+
// If a node has multiple outputs it should emit a new counter value to its inputs
209+
// whenever any of its outputs changes which means the counters sent to inputs may be
210+
// larger than the counters received on its outputs.
211+
//
212+
// A node with multiple outputs will also need to ensure it is applying backpressure if
213+
// any of its outputs is asking to pause
214+
197215
/// \brief Start producing
198216
///
199217
/// This must only be called once. If this fails, then other lifecycle
@@ -204,22 +222,26 @@ class ARROW_EXPORT ExecNode {
204222

205223
/// \brief Pause producing temporarily
206224
///
225+
/// \param output Pointer to the output that is full
226+
/// \param counter Counter used to sequence calls to pause/resume
227+
///
207228
/// This call is a hint that an output node is currently not willing
208229
/// to receive data.
209230
///
210231
/// This may be called any number of times after StartProducing() succeeds.
211232
/// However, the node is still free to produce data (which may be difficult
212233
/// to prevent anyway if data is produced using multiple threads).
213-
virtual void PauseProducing(ExecNode* output) = 0;
234+
virtual void PauseProducing(ExecNode* output, int32_t counter) = 0;
214235

215236
/// \brief Resume producing after a temporary pause
216237
///
238+
/// \param output Pointer to the output that is now free
239+
/// \param counter Counter used to sequence calls to pause/resume
240+
///
217241
/// This call is a hint that an output node is willing to receive data again.
218242
///
219243
/// This may be called any number of times after StartProducing() succeeds.
220-
/// This may also be called concurrently with PauseProducing(), which suggests
221-
/// the implementation may use an atomic counter.
222-
virtual void ResumeProducing(ExecNode* output) = 0;
244+
virtual void ResumeProducing(ExecNode* output, int32_t counter) = 0;
223245

224246
/// \brief Stop producing definitively to a single output
225247
///
@@ -281,9 +303,9 @@ class MapNode : public ExecNode {
281303

282304
Status StartProducing() override;
283305

284-
void PauseProducing(ExecNode* output) override;
306+
void PauseProducing(ExecNode* output, int32_t counter) override;
285307

286-
void ResumeProducing(ExecNode* output) override;
308+
void ResumeProducing(ExecNode* output, int32_t counter) override;
287309

288310
void StopProducing(ExecNode* output) override;
289311

cpp/src/arrow/compute/exec/hash_join_node.cc

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -592,9 +592,13 @@ class HashJoinNode : public ExecNode {
592592
return Status::OK();
593593
}
594594

595-
void PauseProducing(ExecNode* output) override { EVENT(span_, "PauseProducing"); }
595+
void PauseProducing(ExecNode* output, int32_t counter) override {
596+
// TODO(ARROW-16246)
597+
}
596598

597-
void ResumeProducing(ExecNode* output) override { EVENT(span_, "ResumeProducing"); }
599+
void ResumeProducing(ExecNode* output, int32_t counter) override {
600+
// TODO(ARROW-16246)
601+
}
598602

599603
void StopProducing(ExecNode* output) override {
600604
DCHECK_EQ(output, outputs_[0]);

cpp/src/arrow/compute/exec/options.h

Lines changed: 73 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -129,17 +129,85 @@ class ARROW_EXPORT AggregateNodeOptions : public ExecNodeOptions {
129129
std::vector<FieldRef> keys;
130130
};
131131

132+
constexpr int32_t kDefaultBackpressureHighBytes = 1 << 30; // 1GiB
133+
constexpr int32_t kDefaultBackpressureLowBytes = 1 << 28; // 256MiB
134+
135+
class ARROW_EXPORT BackpressureMonitor {
136+
public:
137+
virtual ~BackpressureMonitor() = default;
138+
virtual uint64_t bytes_in_use() const = 0;
139+
virtual bool is_paused() const = 0;
140+
};
141+
142+
/// \brief Options to control backpressure behavior
143+
struct ARROW_EXPORT BackpressureOptions {
144+
/// \brief Create default options that perform no backpressure
145+
BackpressureOptions() : resume_if_below(0), pause_if_above(0) {}
146+
/// \brief Create options that will perform backpressure
147+
///
148+
/// \param resume_if_below The producer should resume producing if the backpressure
149+
/// queue has fewer than resume_if_below items.
150+
/// \param pause_if_above The producer should pause producing if the backpressure
151+
/// queue has more than pause_if_above items
152+
BackpressureOptions(uint32_t resume_if_below, uint32_t pause_if_above)
153+
: resume_if_below(resume_if_below), pause_if_above(pause_if_above) {}
154+
155+
static BackpressureOptions DefaultBackpressure() {
156+
return BackpressureOptions(kDefaultBackpressureLowBytes,
157+
kDefaultBackpressureHighBytes);
158+
}
159+
160+
bool should_apply_backpressure() const { return pause_if_above > 0; }
161+
162+
uint64_t resume_if_below;
163+
uint64_t pause_if_above;
164+
};
165+
132166
/// \brief Add a sink node which forwards to an AsyncGenerator<ExecBatch>
133167
///
134168
/// Emitted batches will not be ordered.
135169
class ARROW_EXPORT SinkNodeOptions : public ExecNodeOptions {
136170
public:
137171
explicit SinkNodeOptions(std::function<Future<util::optional<ExecBatch>>()>* generator,
138-
util::BackpressureOptions backpressure = {})
139-
: generator(generator), backpressure(std::move(backpressure)) {}
172+
BackpressureOptions backpressure = {},
173+
BackpressureMonitor** backpressure_monitor = NULLPTR)
174+
: generator(generator),
175+
backpressure(std::move(backpressure)),
176+
backpressure_monitor(backpressure_monitor) {}
140177

178+
/// \brief A pointer to a generator of batches.
179+
///
180+
/// This will be set when the node is added to the plan and should be used to consume
181+
/// data from the plan. If this function is not called frequently enough then the sink
182+
/// node will start to accumulate data and may apply backpressure.
141183
std::function<Future<util::optional<ExecBatch>>()>* generator;
142-
util::BackpressureOptions backpressure;
184+
/// \brief Options to control when to apply backpressure
185+
///
186+
/// This is optional, the default is to never apply backpressure. If the plan is not
187+
/// consumed quickly enough the system may eventually run out of memory.
188+
BackpressureOptions backpressure;
189+
/// \brief A pointer to a backpressure monitor
190+
///
191+
/// This will be set when the node is added to the plan. This can be used to inspect
192+
/// the amount of data currently queued in the sink node. This is an optional utility
193+
/// and backpressure can be applied even if this is not used.
194+
BackpressureMonitor** backpressure_monitor;
195+
};
196+
197+
/// \brief Control used by a SinkNodeConsumer to pause & resume
198+
///
199+
/// Callers should ensure that they do not call Pause and Resume simultaneously and they
200+
/// should sequence things so that a call to Pause() is always followed by an eventual
201+
/// call to Resume()
202+
class ARROW_EXPORT BackpressureControl {
203+
public:
204+
/// \brief Ask the input to pause
205+
///
206+
/// This is best effort, batches may continue to arrive
207+
/// Must eventually be followed by a call to Resume() or deadlock will occur
208+
virtual void Pause() = 0;
209+
/// \brief Ask the input to resume
210+
virtual void Resume() = 0;
143211
};
144212

145213
class ARROW_EXPORT SinkNodeConsumer {
@@ -150,7 +218,8 @@ class ARROW_EXPORT SinkNodeConsumer {
150218
/// This will be run once the schema is finalized as the plan is starting and
151219
/// before any calls to Consume. A common use is to save off the schema so that
152220
/// batches can be interpreted.
153-
virtual Status Init(const std::shared_ptr<Schema>& schema) = 0;
221+
virtual Status Init(const std::shared_ptr<Schema>& schema,
222+
BackpressureControl* backpressure_control) = 0;
154223
/// \brief Consume a batch of data
155224
virtual Status Consume(ExecBatch batch) = 0;
156225
/// \brief Signal to the consumer that the last batch has been delivered

0 commit comments

Comments
 (0)