-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
GH-41706: [C++][Acero] Enhance asof_join to work in multi-threaded execution by sequencing input #44083
GH-41706: [C++][Acero] Enhance asof_join to work in multi-threaded execution by sequencing input #44083
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
@@ -1032,11 +1032,11 @@ Result<acero::ExecNode*> MakeScanNode(acero::ExecPlan* plan, | |||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
} else { | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
batch_gen = std::move(merged_batch_gen); | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
int64_t index = require_sequenced_output ? 0 : compute::kUnsequencedIndex; | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
auto gen = MakeMappedGenerator( | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
std::move(batch_gen), | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
[scan_options](const EnumeratedRecordBatch& partial) | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
-> Result<std::optional<compute::ExecBatch>> { | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
[scan_options, index](const EnumeratedRecordBatch& partial) mutable | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
-> Result<std::optional<compute::ExecBatch>> { | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
// TODO(ARROW-13263) fragments may be able to attach more guarantees to batches | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
// than this, for example parquet's row group stats. Failing to do this leaves | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
// perf on the table because row group stats could be used to skip kernel execs in | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
|
@@ -1057,9 +1057,12 @@ Result<acero::ExecNode*> MakeScanNode(acero::ExecPlan* plan, | |||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
batch->values.emplace_back(partial.record_batch.index); | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
batch->values.emplace_back(partial.record_batch.last); | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
batch->values.emplace_back(partial.fragment.value->ToString()); | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
if (index != compute::kUnsequencedIndex) batch->index = index++; | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
return batch; | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
}); | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Looks like you are adding the batch index here, which is already done by the arrow/cpp/src/arrow/acero/source_node.cc Lines 134 to 162 in 567f9c5
This seems redundant. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It seems so. I first added indexing and then fixed ordering information propagation to source_node. Not sure it is possible to update already merged PR. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It's not possible, I'll draft a followup PR, if you don't mind. |
||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
auto ordering = require_sequenced_output ? Ordering::Implicit() : Ordering::Unordered(); | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
auto fields = scan_options->dataset_schema->fields(); | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
if (scan_options->add_augmented_fields) { | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
for (const auto& aug_field : kAugmentedFields) { | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
|
@@ -1069,7 +1072,7 @@ Result<acero::ExecNode*> MakeScanNode(acero::ExecPlan* plan, | |||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
return acero::MakeExecNode( | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
"source", plan, {}, | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
acero::SourceNodeOptions{schema(std::move(fields)), std::move(gen)}); | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
acero::SourceNodeOptions{schema(std::move(fields)), std::move(gen), ordering}); | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
Result<acero::ExecNode*> MakeAugmentedProjectNode(acero::ExecPlan* plan, | ||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -56,8 +56,10 @@ class InMemoryDataset: | |
ds = DatasetModuleStub | ||
|
||
|
||
def _dataset_to_decl(dataset, use_threads=True): | ||
decl = Declaration("scan", ScanNodeOptions(dataset, use_threads=use_threads)) | ||
def _dataset_to_decl(dataset, use_threads=True, require_sequenced_output=False): | ||
decl = Declaration("scan", ScanNodeOptions( | ||
dataset, use_threads=use_threads, | ||
require_sequenced_output=require_sequenced_output)) | ||
|
||
# Get rid of special dataset columns | ||
# "__fragment_index", "__batch_index", "__last_in_fragment", "__filename" | ||
|
@@ -311,13 +313,18 @@ def _perform_join_asof(left_operand, left_on, left_by, | |
|
||
# Add the join node to the execplan | ||
if isinstance(left_operand, ds.Dataset): | ||
left_source = _dataset_to_decl(left_operand, use_threads=use_threads) | ||
left_source = _dataset_to_decl( | ||
left_operand, | ||
use_threads=use_threads, | ||
require_sequenced_output=True) | ||
else: | ||
left_source = Declaration( | ||
"table_source", TableSourceNodeOptions(left_operand), | ||
) | ||
if isinstance(right_operand, ds.Dataset): | ||
right_source = _dataset_to_decl(right_operand, use_threads=use_threads) | ||
right_source = _dataset_to_decl( | ||
right_operand, use_threads=use_threads, | ||
require_sequenced_output=True) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do you really need the dataset to be sequenced? I think all you need is implicit ordering (batch indices) so you can sequence the batches at any point down the pipeline (by your asof join node). There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
True. But I don't think In context of your GH-26818 P.S. I think that the idea of "Dataset has implicit ordering" is not handled in arrow. There is no way to store ordering information in dataset and order is subjected to alphabetical ordering based on fragment filenames(which with default filename template is incorrect |
||
else: | ||
right_source = Declaration( | ||
"table_source", TableSourceNodeOptions(right_operand) | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Isn't this a bit too restrictive? This means all RecordBatch reader source always preserve the implicit order, even if if users do not care? Shouldn't this be configurable? What about other sources? Shouldn't the behaviour be the same for any source?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Probably option to specify ordering would be best. Or maybe automatically assign implicit ordering based on
batch.index!=compute::kUnsequencedIndex
?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can you elaborate on why your asof_join enhancement requires specifically the
RecordBatchReaderSrouceNode
to have implicit ordering?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It does not. I assumed that when user provides a generator, it has some kind of implicot order. Now I see it is not necessarily a case.