-
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
ARROW-12208: [C++] Add the ability to run async tasks without using the CPU thread pool #9892
Changes from all commits
541e494
0d343cf
d26283a
49a89bd
5fc6f94
9859392
fc2aa0c
3e8a346
c215a31
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 |
---|---|---|
|
@@ -43,6 +43,8 @@ namespace dataset { | |
|
||
using internal::checked_cast; | ||
using internal::checked_pointer_cast; | ||
using internal::Executor; | ||
using internal::SerialExecutor; | ||
using RecordBatchGenerator = AsyncGenerator<std::shared_ptr<RecordBatch>>; | ||
|
||
Result<std::unordered_set<std::string>> GetColumnNames( | ||
|
@@ -107,13 +109,14 @@ static inline Result<csv::ReadOptions> GetReadOptions( | |
auto read_options = csv_scan_options->read_options; | ||
// Multithreaded conversion of individual files would lead to excessive thread | ||
// contention when ScanTasks are also executed in multiple threads, so we disable it | ||
// here. | ||
// here. Also, this is a no-op since the streaming CSV reader is currently serial | ||
read_options.use_threads = false; | ||
return read_options; | ||
} | ||
|
||
static inline Future<std::shared_ptr<csv::StreamingReader>> OpenReaderAsync( | ||
const FileSource& source, const CsvFileFormat& format, | ||
internal::Executor* cpu_executor, | ||
const std::shared_ptr<ScanOptions>& scan_options = nullptr, | ||
MemoryPool* pool = default_memory_pool()) { | ||
ARROW_ASSIGN_OR_RAISE(auto reader_options, GetReadOptions(format, scan_options)); | ||
|
@@ -136,7 +139,8 @@ static inline Future<std::shared_ptr<csv::StreamingReader>> OpenReaderAsync( | |
} | ||
|
||
return csv::StreamingReader::MakeAsync(io::default_io_context(), std::move(input), | ||
reader_options, parse_options, convert_options) | ||
cpu_executor, reader_options, parse_options, | ||
convert_options) | ||
.Then( | ||
[](const std::shared_ptr<csv::StreamingReader>& maybe_reader) | ||
-> Result<std::shared_ptr<csv::StreamingReader>> { return maybe_reader; }, | ||
|
@@ -151,8 +155,12 @@ static inline Result<std::shared_ptr<csv::StreamingReader>> OpenReader( | |
const FileSource& source, const CsvFileFormat& format, | ||
const std::shared_ptr<ScanOptions>& scan_options = nullptr, | ||
MemoryPool* pool = default_memory_pool()) { | ||
auto open_reader_fut = OpenReaderAsync(source, format, scan_options, pool); | ||
return open_reader_fut.result(); | ||
bool use_threads = (scan_options != nullptr && scan_options->use_threads); | ||
return internal::RunSynchronously<std::shared_ptr<csv::StreamingReader>>( | ||
[&](Executor* executor) { | ||
return OpenReaderAsync(source, format, executor, scan_options, pool); | ||
}, | ||
use_threads); | ||
} | ||
|
||
/// \brief A ScanTask backed by an Csv file. | ||
|
@@ -166,14 +174,15 @@ class CsvScanTask : public ScanTask { | |
source_(fragment->source()) {} | ||
|
||
Result<RecordBatchIterator> Execute() override { | ||
ARROW_ASSIGN_OR_RAISE(auto gen, ExecuteAsync()); | ||
ARROW_ASSIGN_OR_RAISE(auto gen, ExecuteAsync(internal::GetCpuThreadPool())); | ||
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. Shouldn't you lookup 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. This is a bit awkward. This method would not be used by either In that case we cannot really use I suppose that means there is no truly serial way to call |
||
return MakeGeneratorIterator(std::move(gen)); | ||
} | ||
|
||
bool supports_async() const override { return true; } | ||
|
||
Result<RecordBatchGenerator> ExecuteAsync() override { | ||
auto reader_fut = OpenReaderAsync(source_, *format_, options(), options()->pool); | ||
Result<RecordBatchGenerator> ExecuteAsync(internal::Executor* cpu_executor) override { | ||
auto reader_fut = | ||
OpenReaderAsync(source_, *format_, cpu_executor, options(), options()->pool); | ||
auto generator_fut = reader_fut.Then( | ||
[](const std::shared_ptr<csv::StreamingReader>& reader) -> RecordBatchGenerator { | ||
return [reader]() { return reader->ReadNextAsync(); }; | ||
|
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.
arrow/util/type_fwd.h
was probably sufficient.