From 13da11047ca0367c2e3e8df276e257c79f3bfc8a Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Thu, 9 Mar 2023 13:24:26 -0800 Subject: [PATCH 01/13] Added Parquet support for the new scan node. Refactored the scan node considerably. Now each fragment contains one or more scan tasks. Each scan task can yield a stream of batches. So, CSV, for example, is a single scan task that covers the entire file. Parquet, on the other hand, has a scan task per row group. This also makes explicit a lot of the logic that was implicit around sequencing and trying to figure out the correct batch index. --- cpp/src/arrow/compute/key_map_avx2.cc | 2 + cpp/src/arrow/dataset/dataset.cc | 57 +- cpp/src/arrow/dataset/dataset.h | 97 +++- cpp/src/arrow/dataset/dataset_internal.h | 15 + cpp/src/arrow/dataset/file_base.cc | 13 +- cpp/src/arrow/dataset/file_base.h | 10 +- cpp/src/arrow/dataset/file_csv.cc | 47 +- cpp/src/arrow/dataset/file_csv.h | 3 +- cpp/src/arrow/dataset/file_csv_test.cc | 13 +- cpp/src/arrow/dataset/file_json.cc | 28 +- cpp/src/arrow/dataset/file_json.h | 4 +- cpp/src/arrow/dataset/file_json_test.cc | 4 +- cpp/src/arrow/dataset/file_parquet.cc | 277 +++++++++- cpp/src/arrow/dataset/file_parquet.h | 77 ++- cpp/src/arrow/dataset/file_parquet_test.cc | 22 + cpp/src/arrow/dataset/scan_node.cc | 600 +++++++++++++++------ cpp/src/arrow/dataset/scanner.h | 73 ++- cpp/src/arrow/dataset/scanner_test.cc | 111 +++- cpp/src/arrow/dataset/test_util_internal.h | 229 +++----- cpp/src/arrow/testing/matchers.h | 2 +- cpp/src/arrow/util/async_util.cc | 6 + cpp/src/arrow/util/async_util.h | 32 +- cpp/src/parquet/arrow/reader.cc | 160 ++++++ cpp/src/parquet/arrow/reader.h | 65 +++ 24 files changed, 1445 insertions(+), 502 deletions(-) diff --git a/cpp/src/arrow/compute/key_map_avx2.cc b/cpp/src/arrow/compute/key_map_avx2.cc index eb318ff188fbb..a71150a64e0bb 100644 --- a/cpp/src/arrow/compute/key_map_avx2.cc +++ b/cpp/src/arrow/compute/key_map_avx2.cc @@ -20,6 +20,8 @@ #include "arrow/compute/key_map.h" #include "arrow/util/logging.h" +#include "arrow/util/logging.h" + namespace arrow { namespace compute { diff --git a/cpp/src/arrow/dataset/dataset.cc b/cpp/src/arrow/dataset/dataset.cc index 0e7bc3da84235..c611b297c3715 100644 --- a/cpp/src/arrow/dataset/dataset.cc +++ b/cpp/src/arrow/dataset/dataset.cc @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +#include #include #include @@ -44,13 +45,31 @@ Fragment::Fragment(compute::Expression partition_expression, physical_schema_(std::move(physical_schema)) {} Future> Fragment::InspectFragment( + const FragmentScanOptions* format_options, compute::ExecContext* exec_context, + bool should_cache) { + util::Mutex::Guard lk = physical_schema_mutex_.Lock(); + if (cached_inspected_fragment_) { + return cached_inspected_fragment_; + } + lk.Unlock(); + return InspectFragmentImpl(format_options, exec_context) + .Then([this, should_cache](const std::shared_ptr& frag) { + if (should_cache) { + util::Mutex::Guard lk = physical_schema_mutex_.Lock(); + cached_inspected_fragment_ = frag; + } + return frag; + }); +} + +Future> Fragment::InspectFragmentImpl( const FragmentScanOptions* format_options, compute::ExecContext* exec_context) { return Status::NotImplemented("Inspect fragment"); } Future> Fragment::BeginScan( const FragmentScanRequest& request, const InspectedFragment& inspected_fragment, - const FragmentScanOptions* format_options, compute::ExecContext* exec_context) { + compute::ExecContext* exec_context) { return Status::NotImplemented("New scan method"); } @@ -156,42 +175,44 @@ Future> InMemoryFragment::CountRows( return Future>::MakeFinished(total); } -Future> InMemoryFragment::InspectFragment( +Future> InMemoryFragment::InspectFragmentImpl( const FragmentScanOptions* format_options, compute::ExecContext* exec_context) { return std::make_shared(physical_schema_->field_names()); } class InMemoryFragment::Scanner : public FragmentScanner { public: - explicit Scanner(InMemoryFragment* fragment) : fragment_(fragment) {} + explicit Scanner(std::vector> batches) + : batches_(std::move(batches)) {} - Future> ScanBatch(int batch_number) override { - return Future>::MakeFinished( - fragment_->record_batches_[batch_number]); + AsyncGenerator> RunScanTask(int batch_number) override { + DCHECK_EQ(batch_number, 0); + return MakeVectorGenerator(std::move(batches_)); } - int64_t EstimatedDataBytes(int batch_number) override { - return arrow::util::TotalBufferSize(*fragment_->record_batches_[batch_number]); - } + int NumScanTasks() override { return 1; } - int NumBatches() override { - return static_cast(fragment_->record_batches_.size()); + int NumBatchesInScanTask(int task_number) override { + DCHECK_LE(batches_.size(), std::numeric_limits::max()); + return static_cast(batches_.size()); } private: - InMemoryFragment* fragment_; + std::vector> batches_; }; Future> InMemoryFragment::BeginScan( const FragmentScanRequest& request, const InspectedFragment& inspected_fragment, - const FragmentScanOptions* format_options, compute::ExecContext* exec_context) { + compute::ExecContext* exec_context) { return Future>::MakeFinished( - std::make_shared(this)); + std::make_shared(record_batches_)); } -Dataset::Dataset(std::shared_ptr schema, compute::Expression partition_expression) +Dataset::Dataset(std::shared_ptr schema, compute::Expression partition_expression, + bool should_cache_metadata) : schema_(std::move(schema)), - partition_expression_(std::move(partition_expression)) {} + partition_expression_(std::move(partition_expression)), + should_cache_metadata_(should_cache_metadata) {} Result> Dataset::NewScan() { return std::make_shared(this->shared_from_this()); @@ -246,7 +267,7 @@ struct VectorRecordBatchGenerator : InMemoryDataset::RecordBatchGenerator { InMemoryDataset::InMemoryDataset(std::shared_ptr schema, RecordBatchVector batches) - : Dataset(std::move(schema)), + : Dataset(std::move(schema), /*should_cache_metadata=*/false), get_batches_(new VectorRecordBatchGenerator(std::move(batches))) {} struct TableRecordBatchGenerator : InMemoryDataset::RecordBatchGenerator { @@ -263,7 +284,7 @@ struct TableRecordBatchGenerator : InMemoryDataset::RecordBatchGenerator { }; InMemoryDataset::InMemoryDataset(std::shared_ptr table) - : Dataset(table->schema()), + : Dataset(table->schema(), /*should_cache_metadata=*/false), get_batches_(new TableRecordBatchGenerator(std::move(table))) {} Result> InMemoryDataset::ReplaceSchema( diff --git a/cpp/src/arrow/dataset/dataset.h b/cpp/src/arrow/dataset/dataset.h index 1db230b16e9c2..48e57203454fd 100644 --- a/cpp/src/arrow/dataset/dataset.h +++ b/cpp/src/arrow/dataset/dataset.h @@ -109,25 +109,58 @@ struct ARROW_DS_EXPORT FragmentScanRequest { const FragmentScanOptions* format_scan_options; }; -/// \brief An iterator-like object that can yield batches created from a fragment +/// \brief An abstraction over (potentially parallel) reading of a fragment class ARROW_DS_EXPORT FragmentScanner { public: - /// This instance will only be destroyed after all ongoing scan futures + /// This instance will only be destroyed after all ongoing scan tasks /// have been completed. /// /// This means any callbacks created as part of the scan can safely /// capture `this` virtual ~FragmentScanner() = default; - /// \brief Scan a batch of data from the file - /// \param batch_number The index of the batch to read - virtual Future> ScanBatch(int batch_number) = 0; - /// \brief Calculate an estimate of how many data bytes the given batch will represent + /// \brief Run a task to scan a batches of data from a file /// - /// "Data bytes" should be the total size of all the buffers once the data has been - /// decoded into the Arrow format. - virtual int64_t EstimatedDataBytes(int batch_number) = 0; - /// \brief The number of batches in the fragment to scan - virtual int NumBatches() = 0; + /// Each scan task will generate a sequence of batches. If a file supports multiple + /// scan tasks then the scan tasks should be able to run in parallel. + /// + /// For example, the CSV scanner currently generates a single stream of batches from + /// the start of the file to the end. It is not capable of reading batches in parallel + /// and so there is a single scan task. + /// + /// The parquet scanner can read from different row groups concurrently. Each row group + /// generates a sequence of batches (row groups can be very large and we may not want + /// to read the row group into memory all at once). + /// + /// Multiple scan tasks will be launched in parallel. In other words, RunScanTask + /// will be called async-reentrantly (it will be called again before the future it + /// returns finishes) + /// + /// However, RunScanTask will not be called sync-reentrantly (it will not be + /// called again while a call to this method is in progress) and it will be called + /// in order. + /// + /// For example, RunScanTask(5) will always be called after RunScanTask(4) yet the + /// batches from scan task 4 may arrive before the batches from scan task 5 and this is + /// ok. If the user desires ordered execution then batches will be sequenced later. + /// + /// \param task_number The index of the scan task to execute + virtual AsyncGenerator> RunScanTask(int task_number) = 0; + + /// \brief The total number of scan tasks that will be run + virtual int NumScanTasks() = 0; + + static constexpr int kUnknownNumberOfBatches = -1; + /// \brief The total number of batches that will be delivered by a scan task + /// + /// Ideally, this will be known in advance by inspecting the metadata. A fragment + /// scanner may choose to emit empty batches in order to respect this value. + /// + /// If it is not possible to know this in advance, then a fragment may return + /// FragmentScanner::kUnknownNumberOfBatches. Note that doing so will have a + /// significant negative effect on scan parallelism because a scan task will not start + /// until we have determined how many batches precede it. This means that any scan + /// tasks following this one will have to wait until this scan task is fully exhausted. + virtual int NumBatchesInScanTask(int task_number) = 0; }; /// \brief Information learned about a fragment through inspection @@ -140,8 +173,11 @@ class ARROW_DS_EXPORT FragmentScanner { /// names and use those column names to determine which columns to load /// from the CSV file. struct ARROW_DS_EXPORT InspectedFragment { + virtual ~InspectedFragment() = default; + explicit InspectedFragment(std::vector column_names) : column_names(std::move(column_names)) {} + std::vector column_names; }; @@ -175,12 +211,13 @@ class ARROW_DS_EXPORT Fragment : public std::enable_shared_from_this { /// information will be needed to figure out an evolution strategy. This information /// will then be passed to the call to BeginScan virtual Future> InspectFragment( - const FragmentScanOptions* format_options, compute::ExecContext* exec_context); + const FragmentScanOptions* format_options, compute::ExecContext* exec_context, + bool should_cache); /// \brief Start a scan operation virtual Future> BeginScan( const FragmentScanRequest& request, const InspectedFragment& inspected_fragment, - const FragmentScanOptions* format_options, compute::ExecContext* exec_context); + compute::ExecContext* exec_context); /// \brief Count the number of rows in this fragment matching the filter using metadata /// only. That is, this method may perform I/O, but will not load data. @@ -206,11 +243,14 @@ class ARROW_DS_EXPORT Fragment : public std::enable_shared_from_this { explicit Fragment(compute::Expression partition_expression, std::shared_ptr physical_schema); + virtual Future> InspectFragmentImpl( + const FragmentScanOptions* format_options, compute::ExecContext* exec_context); virtual Result> ReadPhysicalSchemaImpl() = 0; util::Mutex physical_schema_mutex_; compute::Expression partition_expression_ = compute::literal(true); std::shared_ptr physical_schema_; + std::shared_ptr cached_inspected_fragment_; }; /// \brief Per-scan options for fragment(s) in a dataset. @@ -248,12 +288,11 @@ class ARROW_DS_EXPORT InMemoryFragment : public Fragment { compute::Expression predicate, const std::shared_ptr& options) override; - Future> InspectFragment( + Future> InspectFragmentImpl( const FragmentScanOptions* format_options, compute::ExecContext* exec_context) override; Future> BeginScan( const FragmentScanRequest& request, const InspectedFragment& inspected_fragment, - const FragmentScanOptions* format_options, compute::ExecContext* exec_context) override; std::string type_name() const override { return "in-memory"; } @@ -348,6 +387,19 @@ MakeBasicDatasetEvolutionStrategy(); /// A Dataset acts as a union of Fragments, e.g. files deeply nested in a /// directory. A Dataset has a schema to which Fragments must align during a /// scan operation. This is analogous to Avro's reader and writer schema. +/// +/// It is assumed that a dataset will always generate fragments in the same +/// order. Data in a dataset thus has an "implicit order" which is first +/// decided by the fragment index and then the row index in a fragment. For +/// example, row 1 in fragment 10 comes after the last row in fragment 9. +/// +/// A dataset will cache metadata by default. This will enable future scans +/// to be faster since they can skip some of the initial read steps. However, +/// if the dataset has many files, or if the file metadata itself is large, this +/// cached metadata could occupy a large amount of RAM. +/// +/// Metadata should not be cached if the contents of the files are expected +/// to change between scans. class ARROW_DS_EXPORT Dataset : public std::enable_shared_from_this { public: /// \brief Begin to build a new Scan operation against this Dataset @@ -385,9 +437,15 @@ class ARROW_DS_EXPORT Dataset : public std::enable_shared_from_this { virtual ~Dataset() = default; protected: - explicit Dataset(std::shared_ptr schema) : schema_(std::move(schema)) {} + /// \brief Create a new dataset + /// \param schema the dataset schema. This is the unified schema across all fragments + /// \param should_cache_metadata if true then this dataset instance should try and cache + /// metadata information during a scan. + explicit Dataset(std::shared_ptr schema, bool should_cache_metadata = true) + : schema_(std::move(schema)), should_cache_metadata_(should_cache_metadata) {} - Dataset(std::shared_ptr schema, compute::Expression partition_expression); + Dataset(std::shared_ptr schema, compute::Expression partition_expression, + bool should_cache_metadata = true); virtual Result GetFragmentsImpl(compute::Expression predicate) = 0; /// \brief Default non-virtual implementation method for the base @@ -405,6 +463,8 @@ class ARROW_DS_EXPORT Dataset : public std::enable_shared_from_this { std::shared_ptr schema_; compute::Expression partition_expression_ = compute::literal(true); + bool should_cache_metadata_; + std::unique_ptr evolution_strategy_ = MakeBasicDatasetEvolutionStrategy(); }; @@ -427,7 +487,8 @@ class ARROW_DS_EXPORT InMemoryDataset : public Dataset { /// Construct a dataset from a schema and a factory of record batch iterators. InMemoryDataset(std::shared_ptr schema, std::shared_ptr get_batches) - : Dataset(std::move(schema)), get_batches_(std::move(get_batches)) {} + : Dataset(std::move(schema), /*should_cache_metadata=*/false), + get_batches_(std::move(get_batches)) {} /// Convenience constructor taking a fixed list of batches InMemoryDataset(std::shared_ptr schema, RecordBatchVector batches); diff --git a/cpp/src/arrow/dataset/dataset_internal.h b/cpp/src/arrow/dataset/dataset_internal.h index aac437df0755b..28655a90ed245 100644 --- a/cpp/src/arrow/dataset/dataset_internal.h +++ b/cpp/src/arrow/dataset/dataset_internal.h @@ -89,6 +89,21 @@ arrow::Result> GetFragmentScanOptions( return ::arrow::internal::checked_pointer_cast(source); } +template +Result GetFragmentScanOptions(const FragmentScanOptions* scan_or_format_opts, + const std::string& type_name) { + static T fallback_opts = {}; + if (scan_or_format_opts == nullptr) { + return &fallback_opts; + } + auto* casted_opts = dynamic_cast(scan_or_format_opts); + if (!casted_opts) { + return Status::Invalid("User provided scan options of type ", + scan_or_format_opts->type_name(), " but expected ", type_name); + } + return casted_opts; +} + class FragmentDataset : public Dataset { public: FragmentDataset(std::shared_ptr schema, FragmentVector fragments) diff --git a/cpp/src/arrow/dataset/file_base.cc b/cpp/src/arrow/dataset/file_base.cc index 2fcd57d2f3622..e3e092052d3a9 100644 --- a/cpp/src/arrow/dataset/file_base.cc +++ b/cpp/src/arrow/dataset/file_base.cc @@ -140,7 +140,8 @@ Future> FileFormat::InspectFragment( } Future> FileFormat::BeginScan( - const FragmentScanRequest& request, const InspectedFragment& inspected_fragment, + const FileSource& source, const FragmentScanRequest& request, + const InspectedFragment& inspected_fragment, const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const { return Status::NotImplemented("This format does not yet support the scan2 node"); } @@ -174,7 +175,7 @@ Result FileFragment::ScanBatchesAsync( return format_->ScanBatchesAsync(options, self); } -Future> FileFragment::InspectFragment( +Future> FileFragment::InspectFragmentImpl( const FragmentScanOptions* format_options, compute::ExecContext* exec_context) { const FragmentScanOptions* realized_format_options = format_options; if (format_options == nullptr) { @@ -185,12 +186,12 @@ Future> FileFragment::InspectFragment( Future> FileFragment::BeginScan( const FragmentScanRequest& request, const InspectedFragment& inspected_fragment, - const FragmentScanOptions* format_options, compute::ExecContext* exec_context) { - const FragmentScanOptions* realized_format_options = format_options; - if (format_options == nullptr) { + compute::ExecContext* exec_context) { + const FragmentScanOptions* realized_format_options = request.format_scan_options; + if (realized_format_options == nullptr) { realized_format_options = format_->default_fragment_scan_options.get(); } - return format_->BeginScan(request, inspected_fragment, realized_format_options, + return format_->BeginScan(source_, request, inspected_fragment, realized_format_options, exec_context); } diff --git a/cpp/src/arrow/dataset/file_base.h b/cpp/src/arrow/dataset/file_base.h index d33d88e9966fe..b46c8a3620693 100644 --- a/cpp/src/arrow/dataset/file_base.h +++ b/cpp/src/arrow/dataset/file_base.h @@ -176,8 +176,13 @@ class ARROW_DS_EXPORT FileFormat : public std::enable_shared_from_this& file, compute::Expression predicate, const std::shared_ptr& options); + // `format_options` may seem redundant (there is a `format_options` in `request`) + // however, it will overwrite the scan request's format options with the file format's + // default options if the scan request does not specify `format_options`. So the + // `format_options` parameter should be preferred over `request.format_scan_options` virtual Future> BeginScan( - const FragmentScanRequest& request, const InspectedFragment& inspected_fragment, + const FileSource& source, const FragmentScanRequest& request, + const InspectedFragment& inspected_fragment, const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const; @@ -222,9 +227,8 @@ class ARROW_DS_EXPORT FileFragment : public Fragment, const std::shared_ptr& options) override; Future> BeginScan( const FragmentScanRequest& request, const InspectedFragment& inspected_fragment, - const FragmentScanOptions* format_options, compute::ExecContext* exec_context) override; - Future> InspectFragment( + Future> InspectFragmentImpl( const FragmentScanOptions* format_options, compute::ExecContext* exec_context) override; diff --git a/cpp/src/arrow/dataset/file_csv.cc b/cpp/src/arrow/dataset/file_csv.cc index 09ab775727c98..7e22987c794d3 100644 --- a/cpp/src/arrow/dataset/file_csv.cc +++ b/cpp/src/arrow/dataset/file_csv.cc @@ -67,25 +67,19 @@ struct CsvInspectedFragment : public InspectedFragment { class CsvFileScanner : public FragmentScanner { public: - CsvFileScanner(std::shared_ptr reader, int num_batches, - int64_t best_guess_bytes_per_batch) - : reader_(std::move(reader)), - num_batches_(num_batches), - best_guess_bytes_per_batch_(best_guess_bytes_per_batch) {} - - Future> ScanBatch(int batch_number) override { - // This should be called in increasing order but let's verify that in case it changes. - // It would be easy enough to handle out of order but no need for that complexity at - // the moment. - DCHECK_EQ(scanned_so_far_++, batch_number); - return reader_->ReadNextAsync(); - } + explicit CsvFileScanner(std::shared_ptr reader, int num_batches) + : reader_(std::move(reader)), num_batches_(num_batches) {} - int64_t EstimatedDataBytes(int batch_number) override { - return best_guess_bytes_per_batch_; + AsyncGenerator> RunScanTask(int task_number) override { + return [this] { return reader_->ReadNextAsync(); }; } - int NumBatches() override { return num_batches_; } + int NumScanTasks() override { return 1; } + + int NumBatchesInScanTask(int32_t task_number) override { + DCHECK_EQ(task_number, 0); + return num_batches_; + } static Result GetConvertOptions( const CsvFragmentScanOptions& csv_options, const FragmentScanRequest& scan_request, @@ -114,11 +108,10 @@ class CsvFileScanner : public FragmentScanner { const CsvInspectedFragment& inspected_fragment, Executor* cpu_executor) { auto read_options = csv_options.read_options; + // Right now we disallow values larger than a block. If we wanted to allow them + // however we could just emit an empty batch. So this should be pretty safe. int num_batches = static_cast(bit_util::CeilDiv( inspected_fragment.num_bytes, static_cast(read_options.block_size))); - // Could be better, but a reasonable starting point. CSV presumably takes up more - // space than an in-memory format so this should be conservative. - int64_t best_guess_bytes_per_batch = read_options.block_size; ARROW_ASSIGN_OR_RAISE( csv::ConvertOptions convert_options, GetConvertOptions(csv_options, scan_request, inspected_fragment)); @@ -126,20 +119,15 @@ class CsvFileScanner : public FragmentScanner { return csv::StreamingReader::MakeAsync( io::default_io_context(), inspected_fragment.input_stream, cpu_executor, read_options, csv_options.parse_options, convert_options) - .Then([num_batches, best_guess_bytes_per_batch]( - const std::shared_ptr& reader) + .Then([num_batches](const std::shared_ptr& reader) -> std::shared_ptr { - return std::make_shared(reader, num_batches, - best_guess_bytes_per_batch); + return std::make_shared(reader, num_batches); }); } private: std::shared_ptr reader_; int num_batches_; - int64_t best_guess_bytes_per_batch_; - - int scanned_so_far_ = 0; }; using RecordBatchGenerator = std::function>()>; @@ -412,7 +400,8 @@ Future> CsvFileFormat::CountRows( } Future> CsvFileFormat::BeginScan( - const FragmentScanRequest& request, const InspectedFragment& inspected_fragment, + const FileSource& file_source, const FragmentScanRequest& request, + const InspectedFragment& inspected_fragment, const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const { auto csv_options = static_cast(format_options); auto csv_fragment = static_cast(inspected_fragment); @@ -445,7 +434,9 @@ Result> DoInspectFragment( Future> CsvFileFormat::InspectFragment( const FileSource& source, const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const { - auto csv_options = static_cast(format_options); + ARROW_ASSIGN_OR_RAISE( + const auto* csv_options, + GetFragmentScanOptions(format_options, kCsvTypeName)); Executor* io_executor; if (source.filesystem()) { io_executor = source.filesystem()->io_context().executor(); diff --git a/cpp/src/arrow/dataset/file_csv.h b/cpp/src/arrow/dataset/file_csv.h index 42e3fd7246988..a05a3cb479f71 100644 --- a/cpp/src/arrow/dataset/file_csv.h +++ b/cpp/src/arrow/dataset/file_csv.h @@ -57,7 +57,8 @@ class ARROW_DS_EXPORT CsvFileFormat : public FileFormat { Result> Inspect(const FileSource& source) const override; Future> BeginScan( - const FragmentScanRequest& request, const InspectedFragment& inspected_fragment, + const FileSource& file_source, const FragmentScanRequest& request, + const InspectedFragment& inspected_fragment, const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const override; diff --git a/cpp/src/arrow/dataset/file_csv_test.cc b/cpp/src/arrow/dataset/file_csv_test.cc index 755b202439be6..73899d668682d 100644 --- a/cpp/src/arrow/dataset/file_csv_test.cc +++ b/cpp/src/arrow/dataset/file_csv_test.cc @@ -531,18 +531,13 @@ class TestCsvFileFormatScanNode : public FileFormatScanNodeMixin> ScanBatch(int index) override { - DCHECK_EQ(num_scanned_++, index); - return reader_->ReadNextAsync(); + int NumBatchesInScanTask(int task_number) override { + DCHECK_EQ(task_number, 0); + return num_batches_; } - int64_t EstimatedDataBytes(int) override { return block_size_; } + AsyncGenerator> RunScanTask(int task_number) override { + DCHECK_EQ(task_number, 0); + return [this] { return reader_->ReadNextAsync(); }; + } static Result> GetSchema( const FragmentScanRequest& scan_request, const JsonInspectedFragment& inspected) { @@ -114,17 +117,15 @@ class JsonFragmentScanner : public FragmentScanner { auto future = json::StreamingReader::MakeAsync( inspected.stream, format_options.read_options, parse_options, io::default_io_context(), cpu_executor); - return future.Then([num_batches, block_size](const ReaderPtr& reader) + return future.Then([num_batches](const ReaderPtr& reader) -> Result> { - return std::make_shared(reader, num_batches, block_size); + return std::make_shared(reader, num_batches); }); } private: ReaderPtr reader_; - int64_t block_size_; int num_batches_; - int num_scanned_ = 0; }; // Return the same parse options, but disable any options that could interfere with @@ -412,8 +413,9 @@ Future> JsonFileFormat::InspectFragment( } Future> JsonFileFormat::BeginScan( - const FragmentScanRequest& scan_request, const InspectedFragment& inspected, - const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const { + const FileSource& file_source, const FragmentScanRequest& scan_request, + const InspectedFragment& inspected, const FragmentScanOptions* format_options, + compute::ExecContext* exec_context) const { return JsonFragmentScanner::Make( scan_request, static_cast(*format_options), static_cast(inspected), exec_context->executor()); diff --git a/cpp/src/arrow/dataset/file_json.h b/cpp/src/arrow/dataset/file_json.h index 4b8112d87095c..d331cbdd98a44 100644 --- a/cpp/src/arrow/dataset/file_json.h +++ b/cpp/src/arrow/dataset/file_json.h @@ -58,8 +58,8 @@ class ARROW_DS_EXPORT JsonFileFormat : public FileFormat { compute::ExecContext* exec_context) const override; Future> BeginScan( - const FragmentScanRequest& scan_request, const InspectedFragment& inspected, - const FragmentScanOptions* format_options, + const FileSource& file_source, const FragmentScanRequest& scan_request, + const InspectedFragment& inspected, const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const override; Result ScanBatchesAsync( diff --git a/cpp/src/arrow/dataset/file_json_test.cc b/cpp/src/arrow/dataset/file_json_test.cc index 2b4fcdd82f7ff..3b6c2b337a02d 100644 --- a/cpp/src/arrow/dataset/file_json_test.cc +++ b/cpp/src/arrow/dataset/file_json_test.cc @@ -327,9 +327,7 @@ INSTANTIATE_TEST_SUITE_P(TestJsonScan, TestJsonScan, // Common tests for new scanner TEST_P(TestJsonScanNode, Scan) { TestScan(); } -TEST_P(TestJsonScanNode, ScanMissingFilterField) { TestScanMissingFilterField(); } -TEST_P(TestJsonScanNode, ScanProjected) { TestScanProjected(); } -TEST_P(TestJsonScanNode, ScanProjectedMissingColumns) { TestScanProjectedMissingCols(); } +TEST_P(TestJsonScanNode, ScanSomeColumns) { TestScanSomeColumns(); } // JSON-specific tests for new scanner TEST_P(TestJsonScanNode, ScanWithBOM) { TestScanWithBOM(); } TEST_P(TestJsonScanNode, ScanWithCustomParseOptions) { TestScanWithCustomParseOptions(); } diff --git a/cpp/src/arrow/dataset/file_parquet.cc b/cpp/src/arrow/dataset/file_parquet.cc index c30441d911e4e..fec2bf1e7bb38 100644 --- a/cpp/src/arrow/dataset/file_parquet.cc +++ b/cpp/src/arrow/dataset/file_parquet.cc @@ -17,6 +17,7 @@ #include "arrow/dataset/file_parquet.h" +#include #include #include #include @@ -24,17 +25,26 @@ #include #include +#include "arrow/acero/exec_plan.h" #include "arrow/compute/exec.h" +#include "arrow/dataset/dataset.h" #include "arrow/dataset/dataset_internal.h" #include "arrow/dataset/scanner.h" +#include "arrow/dataset/type_fwd.h" #include "arrow/filesystem/path_util.h" +#include "arrow/io/caching.h" +#include "arrow/io/interfaces.h" +#include "arrow/io/type_fwd.h" #include "arrow/table.h" +#include "arrow/util/async_generator_fwd.h" +#include "arrow/util/bit_util.h" #include "arrow/util/checked_cast.h" #include "arrow/util/future.h" #include "arrow/util/iterator.h" #include "arrow/util/logging.h" #include "arrow/util/range.h" #include "arrow/util/tracing_internal.h" +#include "metadata.h" #include "parquet/arrow/reader.h" #include "parquet/arrow/schema.h" #include "parquet/arrow/writer.h" @@ -56,8 +66,26 @@ using parquet::arrow::StatisticsAsScalars; namespace { +std::vector ColumnNamesFromMetadata( + const parquet::FileMetaData& file_metadata) { + std::vector names(file_metadata.num_columns()); + for (int i = 0; i < file_metadata.num_columns(); i++) { + names[i] = file_metadata.schema()->Column(i)->name(); + } + return names; +} + +class ParquetInspectedFragment : public InspectedFragment { + public: + explicit ParquetInspectedFragment(std::shared_ptr file_metadata) + : InspectedFragment(ColumnNamesFromMetadata(*file_metadata)), + file_metadata(std::move(file_metadata)) {} + + std::shared_ptr file_metadata; +}; + parquet::ReaderProperties MakeReaderProperties( - const ParquetFileFormat& format, ParquetFragmentScanOptions* parquet_scan_options, + const ParquetFragmentScanOptions* parquet_scan_options, MemoryPool* pool = default_memory_pool()) { // Can't mutate pool after construction parquet::ReaderProperties properties(pool); @@ -77,14 +105,14 @@ parquet::ReaderProperties MakeReaderProperties( } parquet::ArrowReaderProperties MakeArrowReaderProperties( - const ParquetFileFormat& format, const parquet::FileMetaData& metadata) { + const ParquetFileFormat::ReaderOptions& reader_options, + const parquet::FileMetaData& metadata) { parquet::ArrowReaderProperties properties(/* use_threads = */ false); - for (const std::string& name : format.reader_options.dict_columns) { + for (const std::string& name : reader_options.dict_columns) { auto column_index = metadata.schema()->ColumnIndex(name); properties.set_read_dictionary(column_index, true); } - properties.set_coerce_int96_timestamp_unit( - format.reader_options.coerce_int96_timestamp_unit); + properties.set_coerce_int96_timestamp_unit(reader_options.coerce_int96_timestamp_unit); return properties; } @@ -298,7 +326,7 @@ Result IsSupportedParquetFile(const ParquetFileFormat& format, GetFragmentScanOptions( kParquetTypeName, nullptr, format.default_fragment_scan_options)); auto reader = parquet::ParquetFileReader::Open( - std::move(input), MakeReaderProperties(format, parquet_scan_options.get())); + std::move(input), MakeReaderProperties(parquet_scan_options.get())); std::shared_ptr metadata = reader->metadata(); return metadata != nullptr && metadata->can_decompress(); } catch (const ::parquet::ParquetInvalidOrCorruptedFileException& e) { @@ -308,6 +336,203 @@ Result IsSupportedParquetFile(const ParquetFileFormat& format, END_PARQUET_CATCH_EXCEPTIONS } +class ParquetFragmentScanner : public FragmentScanner { + public: + explicit ParquetFragmentScanner( + std::string path, std::shared_ptr file_metadata, + const ParquetFragmentScanOptions* scan_options, + const ParquetFileFormat::ReaderOptions* format_reader_options, + compute::ExecContext* exec_context) + : path_(std::move(path)), + file_metadata_(std::move(file_metadata)), + scan_options_(scan_options), + format_reader_options_(format_reader_options), + exec_context_(exec_context) {} + + AsyncGenerator> RunScanTask(int task_number) override { + // TODO(weston) limit columns to read + return file_reader_->ReadRowGroupAsync(task_number, exec_context_->executor()); + } + + int NumScanTasks() override { return file_metadata_->num_row_groups(); } + + int NumBatchesInScanTask(int task_number) override { + if (scan_options_->allow_jumbo_values) { + return FragmentScanner::kUnknownNumberOfBatches; + } + int64_t num_rows = file_metadata_->RowGroup(task_number)->num_rows(); + int64_t num_batches = bit_util::CeilDiv(num_rows, static_cast(batch_size_)); + return static_cast(num_batches); + } + + // These are the parquet-specific reader properties. Some of these properties are set + // from the plan (e.g. memory pool) and some of these properties are controlled by the + // user via ParquetFragmentScanOptions + parquet::ReaderProperties MakeParquetReaderProperties() { + parquet::ReaderProperties reader_properties; + parquet::ReaderProperties properties(exec_context_->memory_pool()); + switch (scan_options_->scan_strategy) { + case ParquetScanStrategy::kLeastMemory: + // This makes it so we don't need to load the entire row group into memory + // but introduces a memcpy of the I/O data, In theory we should be able to prevent + // this memcpy if it is becoming a bottleneck + properties.enable_buffered_stream(); + // 8MiB for reads tends to be a pretty safe balance between too many small reads + // and too few large reads. + properties.set_buffer_size(8 * 1024 * 1024); + break; + case ParquetScanStrategy::kMaxSpeed: + // I'm not actually convinced this does lead to better performance. More + // experiments are needed. However, the way this is documented, it appears that + // should be the case. + properties.disable_buffered_stream(); + break; + case ParquetScanStrategy::kCustom: + // Use what the user provided if custom + if (scan_options_->reader_properties->is_buffered_stream_enabled()) { + properties.enable_buffered_stream(); + } else { + properties.disable_buffered_stream(); + } + properties.set_buffer_size(scan_options_->reader_properties->buffer_size()); + break; + } + // These properties are unrelated to a RAM / CPU tradeoff and we always use what the + // user provided. + properties.file_decryption_properties( + scan_options_->reader_properties->file_decryption_properties()); + properties.set_thrift_string_size_limit( + scan_options_->reader_properties->thrift_string_size_limit()); + properties.set_thrift_container_size_limit( + scan_options_->reader_properties->thrift_container_size_limit()); + return properties; + } + + // These are properties that control how we convert from the Arrow-unaware low level + // parquet reader to the arrow aware reader. Similar to the reader properties these + // come both from the plan itself as well as from user options + // + // In addition, and regrettably, some options come from the format object itself. + // TODO(GH-35211): Simplify this so all options come from ParquetFragmentScanOptions + parquet::ArrowReaderProperties MakeParquetArrowReaderProperties() { + parquet::ArrowReaderProperties properties; + + // These properties are controlled by the plan + properties.set_io_context(io::default_io_context()); + + // These properties are controlled by the user, but simplified + switch (scan_options_->scan_strategy) { + // There's not much point in reading large batches that will just get sliced up by + // the source node anyways. + case ParquetScanStrategy::kLeastMemory: + properties.set_batch_size(acero::ExecPlan::kMaxBatchSize); + // Pre-buffering requires reading the entire row-group all at once + properties.set_pre_buffer(false); + // This does not actually mean we will use threads. The reader's async methods + // take a CPU executor. If that is a serial executor then columns will be + // processed serially. + properties.set_use_threads(true); + break; + case ParquetScanStrategy::kMaxSpeed: + // This batch_size emulates some historical behavior and is probably redundant. + // Since we've disabled stream buffering we are reading entire row groups into + // memory so there isn't all that much reason to have any kind of batch size + properties.set_batch_size(64 * 1024 * 1024); + properties.set_pre_buffer(true); + properties.set_cache_options(io::CacheOptions::LazyDefaults()); + // see comment above about threads + properties.set_use_threads(true); + break; + case ParquetScanStrategy::kCustom: + properties.set_batch_size(scan_options_->arrow_reader_properties->batch_size()); + properties.set_pre_buffer(scan_options_->arrow_reader_properties->pre_buffer()); + properties.set_cache_options( + scan_options_->arrow_reader_properties->cache_options()); + properties.set_use_threads(scan_options_->arrow_reader_properties->use_threads()); + break; + } + + // These options are unrelated to the CPU/RAM tradeoff and we always take from + // the user + for (const std::string& name : format_reader_options_->dict_columns) { + auto column_index = file_metadata_->schema()->ColumnIndex(name); + properties.set_read_dictionary(column_index, true); + } + properties.set_coerce_int96_timestamp_unit( + format_reader_options_->coerce_int96_timestamp_unit); + return properties; + } + + Status CheckRowGroupSizes(int64_t batch_size) { + // This seems extremely unlikely (by default a row group would need 64Bi rows) but + // better safe than sorry since these properties are both int64_t and we would get an + // overflow since we assume int32_t in NumBatchesInScanTask + for (int row_group = 0; row_group < file_metadata_->num_row_groups(); row_group++) { + int64_t num_rows = file_metadata_->RowGroup(row_group)->num_rows(); + if (bit_util::CeilDiv(num_rows, batch_size) > std::numeric_limits::max()) { + return Status::NotImplemented("A single row group with more than 2^31 batches"); + } + } + return Status::OK(); + } + + Future<> Initialize(std::shared_ptr file) { + parquet::ReaderProperties properties = MakeParquetReaderProperties(); + // TODO(ARROW-12259): workaround since we have Future<(move-only type)> + auto reader_fut = parquet::ParquetFileReader::OpenAsync( + std::move(file), std::move(properties), file_metadata_); + return reader_fut.Then( + [this, reader_fut]( + const std::unique_ptr&) mutable -> Status { + ARROW_ASSIGN_OR_RAISE(std::unique_ptr reader, + reader_fut.MoveResult()); + std::shared_ptr metadata = reader->metadata(); + parquet::ArrowReaderProperties arrow_properties = + MakeParquetArrowReaderProperties(); + ARROW_RETURN_NOT_OK(CheckRowGroupSizes(arrow_properties.batch_size())); + if (batch_size_ > std::numeric_limits::max()) { + return Status::NotImplemented("Scanner batch size > int32_t max"); + } + batch_size_ = static_cast(arrow_properties.batch_size()); + RETURN_NOT_OK(parquet::arrow::FileReader::Make( + exec_context_->memory_pool(), std::move(reader), + std::move(arrow_properties), &file_reader_)); + return Status::OK(); + }, + [this](const Status& status) -> Status { + return WrapSourceError(status, path_); + }); + } + + static Future> Make( + std::shared_ptr file, std::string path, + const ParquetFragmentScanOptions* scan_options, + const ParquetFileFormat::ReaderOptions* format_reader_options, + const FragmentScanRequest& request, const ParquetInspectedFragment& inspection, + compute::ExecContext* exec_context) { + // Construct a fragment scanner, initialize it, and return it + std::shared_ptr parquet_fragment_scanner = + std::make_shared(std::move(path), + inspection.file_metadata, scan_options, + format_reader_options, exec_context); + return parquet_fragment_scanner->Initialize(std::move(file)) + .Then([fragment_scanner = std::static_pointer_cast( + parquet_fragment_scanner)]() { return fragment_scanner; }); + } + + private: + // These properties are set during construction + std::string path_; + std::shared_ptr file_metadata_; + const ParquetFragmentScanOptions* scan_options_; + const ParquetFileFormat::ReaderOptions* format_reader_options_; + compute::ExecContext* exec_context_; + + // These are set during Initialize + std::unique_ptr file_reader_; + int32_t batch_size_; +}; + } // namespace std::optional ParquetFileFragment::EvaluateStatisticsAsExpression( @@ -408,6 +633,38 @@ Result> ParquetFileFormat::Inspect( return schema; } +Future> ParquetFileFormat::InspectFragment( + const FileSource& source, const FragmentScanOptions* format_options, + compute::ExecContext* exec_context) const { + ARROW_ASSIGN_OR_RAISE(const ParquetFragmentScanOptions* parquet_scan_options, + GetFragmentScanOptions( + format_options, kParquetTypeName)); + auto properties = + MakeReaderProperties(parquet_scan_options, exec_context->memory_pool()); + ARROW_ASSIGN_OR_RAISE(auto input, source.Open()); + auto reader_fut = + parquet::ParquetFileReader::OpenAsync(std::move(input), std::move(properties)); + return reader_fut.Then([](const std::unique_ptr& reader) + -> std::shared_ptr { + std::shared_ptr metadata = reader->metadata(); + return std::make_shared(std::move(metadata)); + }); +} + +Future> ParquetFileFormat::BeginScan( + const FileSource& source, const FragmentScanRequest& request, + const InspectedFragment& inspected_fragment, + const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const { + ARROW_ASSIGN_OR_RAISE(const ParquetFragmentScanOptions* parquet_scan_options, + GetFragmentScanOptions( + format_options, kParquetTypeName)); + auto inspection = checked_cast(inspected_fragment); + ARROW_ASSIGN_OR_RAISE(auto input, source.Open()); + return ParquetFragmentScanner::Make(std::move(input), source.path(), + parquet_scan_options, &reader_options, request, + inspection, exec_context); +} + Result> ParquetFileFormat::GetReader( const FileSource& source, const std::shared_ptr& options) const { return GetReaderAsync(source, options, nullptr).result(); @@ -431,8 +688,7 @@ Future> ParquetFileFormat::GetReader auto parquet_scan_options, GetFragmentScanOptions(kParquetTypeName, options.get(), default_fragment_scan_options)); - auto properties = - MakeReaderProperties(*this, parquet_scan_options.get(), options->pool); + auto properties = MakeReaderProperties(parquet_scan_options.get(), options->pool); ARROW_ASSIGN_OR_RAISE(auto input, source.Open()); // TODO(ARROW-12259): workaround since we have Future<(move-only type)> auto reader_fut = parquet::ParquetFileReader::OpenAsync( @@ -445,7 +701,8 @@ Future> ParquetFileFormat::GetReader ARROW_ASSIGN_OR_RAISE(std::unique_ptr reader, reader_fut.MoveResult()); std::shared_ptr metadata = reader->metadata(); - auto arrow_properties = MakeArrowReaderProperties(*self, *metadata); + auto arrow_properties = + MakeArrowReaderProperties(self->reader_options, *metadata); arrow_properties.set_batch_size(options->batch_size); // Must be set here since the sync ScanTask handles pre-buffering itself arrow_properties.set_pre_buffer( @@ -933,7 +1190,7 @@ Result> ParquetDatasetFactory::Make( "ParquetDatasetFactory must contain a schema with at least one column"); } - auto properties = MakeArrowReaderProperties(*format, *metadata); + auto properties = MakeArrowReaderProperties(format->reader_options, *metadata); ARROW_ASSIGN_OR_RAISE(auto physical_schema, GetSchema(*metadata, properties)); ARROW_ASSIGN_OR_RAISE(auto manifest, GetSchemaManifest(*metadata, properties)); diff --git a/cpp/src/arrow/dataset/file_parquet.h b/cpp/src/arrow/dataset/file_parquet.h index f33190bd93347..93dc6df458476 100644 --- a/cpp/src/arrow/dataset/file_parquet.h +++ b/cpp/src/arrow/dataset/file_parquet.h @@ -95,6 +95,16 @@ class ARROW_DS_EXPORT ParquetFileFormat : public FileFormat { /// \brief Return the schema of the file if possible. Result> Inspect(const FileSource& source) const override; + Future> InspectFragment( + const FileSource& source, const FragmentScanOptions* format_options, + compute::ExecContext* exec_context) const override; + + Future> BeginScan( + const FileSource& source, const FragmentScanRequest& request, + const InspectedFragment& inspected_fragment, + const FragmentScanOptions* format_options, + compute::ExecContext* exec_context) const override; + Result ScanBatchesAsync( const std::shared_ptr& options, const std::shared_ptr& file) const override; @@ -213,19 +223,76 @@ class ARROW_DS_EXPORT ParquetFileFragment : public FileFragment { friend class ParquetDatasetFactory; }; +enum class ParquetScanStrategy { kLeastMemory, kMaxSpeed, kCustom }; + /// \brief Per-scan options for Parquet fragments class ARROW_DS_EXPORT ParquetFragmentScanOptions : public FragmentScanOptions { public: ParquetFragmentScanOptions(); std::string type_name() const override { return kParquetTypeName; } - /// Reader properties. Not all properties are respected: memory_pool comes from - /// ScanOptions. + /// There are several fine-grained parquet reader properties that are difficult + /// to understand and configure correctly. This often leads to poor performance + /// or poor RAM utilization. + /// + /// As a result, we have created common configurations of these properties. This + /// option allows you to select one of these configurations. + /// + /// kMaxSpeed - This configuration will enable pre-buffering and caching. Row groups + /// will be read in their entirety, batch size is 32Ki. Stream + /// buffering is disabled. + /// + /// kLeastMemory - This configuration will disable pre-buffering. Stream buffering + /// will be enabled. Row groups will be read 32Ki rows at a time. + /// + /// kCustom - Pre-buffering, stream buffering, and batch size settings will be + /// configured according to reader_properties and arrow_reader_properties. + ParquetScanStrategy scan_strategy = ParquetScanStrategy::kMaxSpeed; + + /// Reader properties. + /// + /// Not all properties are respected: + /// + /// * memory_pool comes from ScanOptions. + /// * buffered_stream will be ignored if scan_strategy != kCustom + /// * buffer_size will be ignored if scan_strategy != kCustom std::shared_ptr reader_properties; - /// Arrow reader properties. Not all properties are respected: batch_size comes from - /// ScanOptions. Additionally, dictionary columns come from - /// ParquetFileFormat::ReaderOptions::dict_columns. + /// Arrow reader properties. + /// + /// Not all properties are respected: + /// + /// * batch_size will come from: + /// * ScanOptions if using ScanOptions + /// * ignored if using ScanV2Options and strategy != kCustom (we always use 32Ki) + /// * this arrow_reader_properties if using ScanV2Options and strategy == kCustom + /// * dictionary columns come from ParquetFileFormat::ReaderOptions::dict_columns + /// * pre_buffer will be ignored if using ScanV2Options and scan_strategy != kCustom + /// * cache_options will be ignored if using ScanV2Options and scan_strategy != kCustom + /// * use_threads will come from: + /// * ScanOptions if using ScanOptions + /// * true if using ScanV2Options and the query executor has capacity > 1 + /// * false if using ScanV2Options and the query executor has capacity == 1 std::shared_ptr arrow_reader_properties; + + /// \brief Are parquet files allowed to contain extremely large values + /// + /// The scan will read the parquet file in batches. This is controlled by + /// arrow_reader_properties. It is generally safe to assume that one batch + /// of rows will fit in a RecordBatch. However, if the rows contain string data + /// and the batch size is large, or if there are extremely large strings present + /// in the data, then it may be possible that a single batch of data has more than + /// 2GiB of data in a single column, which is not allowed, and we will need to split + /// the batch. + /// + /// For example, if there is a string column, and the batch size is set to its + /// default value of 32Ki then a batch will be a jumbo if the strings are, on average, + /// 64KiB each. You can avoid this, if needed, by using a custom scan strategy and + /// setting a smaller batch size. + /// + /// If this setting is true it will have a negative impact on performance. We won't + /// know in advance how many batches are contained in the file and this makes it + /// difficult to both scan files in parallel and sequence batches. + bool allow_jumbo_values = false; }; class ARROW_DS_EXPORT ParquetFileWriteOptions : public FileWriteOptions { diff --git a/cpp/src/arrow/dataset/file_parquet_test.cc b/cpp/src/arrow/dataset/file_parquet_test.cc index 42f923f0e6a27..01ca696a431a0 100644 --- a/cpp/src/arrow/dataset/file_parquet_test.cc +++ b/cpp/src/arrow/dataset/file_parquet_test.cc @@ -23,6 +23,7 @@ #include "arrow/compute/api_scalar.h" #include "arrow/dataset/dataset_internal.h" +#include "arrow/dataset/plan.h" #include "arrow/dataset/test_util_internal.h" #include "arrow/io/memory.h" #include "arrow/io/test_common.h" @@ -730,5 +731,26 @@ TEST(TestParquetStatistics, NullMax) { EXPECT_EQ(stat_expression->ToString(), "(x >= 1)"); } +class TestParquetFileFormatScanNode + : public FileFormatScanNodeMixin { + void SetUp() override { internal::Initialize(); } + + const FragmentScanOptions* GetFormatOptions() override { return &scan_options_; } + + protected: + ParquetFragmentScanOptions scan_options_; +}; + +TEST_P(TestParquetFileFormatScanNode, Inspect) { TestInspect(); } +TEST_P(TestParquetFileFormatScanNode, Scan) { TestScan(); } +TEST_P(TestParquetFileFormatScanNode, ScanSomeColumn) { TestScanSomeColumns(); } +TEST_P(TestParquetFileFormatScanNode, ScanWithInvalidOptions) { + TestInvalidFormatScanOptions(); +} + +INSTANTIATE_TEST_SUITE_P(TestScanNode, TestParquetFileFormatScanNode, + ::testing::ValuesIn(TestFormatParams::Values()), + TestFormatParams::ToTestNameString); + } // namespace dataset } // namespace arrow diff --git a/cpp/src/arrow/dataset/scan_node.cc b/cpp/src/arrow/dataset/scan_node.cc index 46cc1dc6c8c3f..dd16a7787d480 100644 --- a/cpp/src/arrow/dataset/scan_node.cc +++ b/cpp/src/arrow/dataset/scan_node.cc @@ -16,10 +16,13 @@ // under the License. #include +#include +#include #include #include #include #include +#include #include #include "arrow/acero/exec_plan.h" @@ -27,11 +30,13 @@ #include "arrow/acero/util.h" #include "arrow/compute/expression.h" #include "arrow/compute/expression_internal.h" +#include "arrow/dataset/dataset.h" #include "arrow/dataset/scanner.h" #include "arrow/record_batch.h" #include "arrow/result.h" #include "arrow/status.h" #include "arrow/type.h" +#include "arrow/util/async_util.h" #include "arrow/util/checked_cast.h" #include "arrow/util/logging.h" #include "arrow/util/string.h" @@ -39,6 +44,7 @@ #include "arrow/util/unreachable.h" using namespace std::string_view_literals; // NOLINT +using namespace std::placeholders; // NOLINT namespace arrow { @@ -48,6 +54,251 @@ namespace dataset { namespace { +// How many inspection tasks we allow to run at the same time +constexpr int kNumConcurrentInspections = 4; +// How many scan tasks we need queued up to pause inspection +constexpr int kLimitQueuedScanTasks = 4; + +// An interface for an object that knows all the details requires to launch a scan task +class ScanTaskLauncher { + public: + virtual ~ScanTaskLauncher() = default; + virtual void LaunchTask(FragmentScanner* scanner, int scan_task_number, + int first_batch_index, + std::function task_complete_cb) = 0; +}; + +// When we have finished inspecting a fragment we can submit its scan tasks. However, +// we only allow certain number of scan tasks at a time. Ironically, a scan task is not +// actually a "task". It's a collection of tasks that will run to scan the stream of +// batches. +// +// This class serves as a synchronous "throttle" which makes sure we aren't running too +// many scan tasks at once. It also pauses the inspection process if we have a bunch +// of scan tasks ready to go. +// +// Lastly, it also serves as a sort of sequencing point where we hold off on launching +// a scan task until we know how many batches precede that scan task. +// +// There are two events. +// +// First, when a fragment finishes inspection, it is inserted into the queue (we might +// know how many batches are in the scan tasks) +// +// * An entry is created for each scan task and added to the queue +// * We check to see if any entries are free to run +// * We may pause or resume the inspection throttle +// +// Second, when a scan task finishes it records that it is finished (at this point we +// definitely know how many batches were in the scan task) +// +// * The task is removed from the queue +// * We check to see if any entries are free to run +// * We may resume the inspection throttle +class ScanTaskStagingArea { + public: + ScanTaskStagingArea(int queue_limit, int run_limit, + util::ThrottledAsyncTaskScheduler* inspection_throttle, + std::function completion_cb) + : queue_limit_(queue_limit), + run_limit_(run_limit), + inspection_throttle_(inspection_throttle), + completion_cb_(std::move(completion_cb)) {} + + void InsertInspectedFragment(std::shared_ptr fragment_scanner, + int fragment_index, + std::unique_ptr task_launcher) { + std::cout << " Inserting fragment into staging area" << std::endl; + auto fragment_entry = std::make_unique( + std::move(fragment_scanner), fragment_index, std::move(task_launcher)); + + std::lock_guard lg(mutex_); + num_queued_scan_tasks_ += static_cast(fragment_entry->scan_tasks.size()); + + // Ordered insertion into linked list + if (!root_) { + root_ = std::move(fragment_entry); + // fragment_entry.prev will be nullptr, which is correct + } else { + FragmentEntry* itr = root_.get(); + while (itr->next != nullptr && itr->next->fragment_index < fragment_index) { + itr = itr->next.get(); + } + if (itr->next != nullptr) { + fragment_entry->next = std::move(itr->next); + fragment_entry->next->prev = fragment_entry.get(); + } + fragment_entry->prev = itr; + itr->next = std::move(fragment_entry); + } + + // Even if this isn't the first fragment it is still possible that there are tasks we + // can now run. For example, if we are allowed to run 4 scan tasks and the root only + // had one. + TryAndLaunchTasksUnlocked(); + + if (num_queued_scan_tasks_ >= queue_limit_) { + inspection_throttle_->Pause(); + } + } + + void FinishedInsertingFragments(int num_fragments) { + std::lock_guard lg(mutex_); + total_num_fragments_ = num_fragments; + if (num_fragments_processed_ == total_num_fragments_) { + std::cout << " Ending on finish" << std::endl; + completion_cb_(); + } + } + + private: + struct ScanTaskEntry { + int scan_task_index; + int num_batches; + bool launched; + }; + struct FragmentEntry { + FragmentEntry(std::shared_ptr fragment_scanner, int fragment_index, + std::unique_ptr task_launcher) + : fragment_scanner(std::move(fragment_scanner)), + fragment_index(fragment_index), + task_launcher(std::move(task_launcher)) { + for (int i = 0; i < this->fragment_scanner->NumScanTasks(); i++) { + ScanTaskEntry scan_task; + scan_task.scan_task_index = i; + scan_task.num_batches = this->fragment_scanner->NumBatchesInScanTask(i); + scan_task.launched = false; + scan_tasks.push_back(scan_task); + } + } + + std::shared_ptr fragment_scanner; + int fragment_index; + std::unique_ptr task_launcher; + + std::unique_ptr next; + FragmentEntry* prev = nullptr; + std::deque scan_tasks; + }; + + void LaunchScanTaskUnlocked(FragmentEntry* entry, int scan_task_number, + int first_batch_index) { + entry->task_launcher->LaunchTask( + entry->fragment_scanner.get(), scan_task_number, first_batch_index, + [this, entry, scan_task_number](int num_batches) { + MarkScanTaskFinished(entry, scan_task_number, num_batches); + }); + num_queued_scan_tasks_--; + if (num_queued_scan_tasks_ < queue_limit_) { + inspection_throttle_->Resume(); + } + num_scan_tasks_running_++; + } + + void TryAndLaunchTasksUnlocked() { + if (!root_) { + std::cout << " Can't launch tasks because there are none" << std::endl; + return; + } + if (num_scan_tasks_running_ >= run_limit_) { + std::cout << " Can't even start trying to run tasks because we hit the limit" + << std::endl; + return; + } + FragmentEntry* itr = root_.get(); + int num_preceding_batches = batches_completed_; + while (itr != nullptr) { + for (auto& scan_task : itr->scan_tasks) { + if (!scan_task.launched) { + std::cout << " Launching a scan task frag=" << itr->fragment_index + << std::endl; + scan_task.launched = true; + LaunchScanTaskUnlocked(itr, scan_task.scan_task_index, num_preceding_batches); + if (num_scan_tasks_running_ >= run_limit_) { + std::cout << " Can't launch more tasks because we hit the limit" + << std::endl; + // We've launched as many as we can + return; + } + } + if (scan_task.num_batches >= 0) { + num_preceding_batches += scan_task.num_batches; + } else { + std::cout << " Can't launch task because waiting for troublemaker" + << std::endl; + // A scan task is running that doesn't know how many batches it has. We can't + // proceed + return; + } + } + itr = itr->next.get(); + } + } + + void MarkScanTaskFinished(FragmentEntry* fragment_entry, int scan_task_number, + int num_batches) { + std::lock_guard lg(mutex_); + batches_completed_ += num_batches; + num_scan_tasks_running_--; + std::cout << " MarkScanTaskFinished(batches_completed_=" << batches_completed_ << ")" + << std::endl; + const auto& itr = fragment_entry->scan_tasks.cbegin(); + std::size_t old_size = fragment_entry->scan_tasks.size(); + while (itr != fragment_entry->scan_tasks.cend()) { + if (itr->scan_task_index == scan_task_number) { + fragment_entry->scan_tasks.erase(itr); + break; + } + } + DCHECK_LT(fragment_entry->scan_tasks.size(), old_size); + if (fragment_entry->scan_tasks.empty()) { + std::cout << "Fragment complete: " << fragment_entry->fragment_index << std::endl; + FragmentEntry* prev = fragment_entry->prev; + if (prev == nullptr) { + // The current root has finished + std::unique_ptr new_root = std::move(root_->next); + if (new_root != nullptr) { + new_root->prev = nullptr; + } + // This next line will cause fragment_entry to be deleted + root_ = std::move(new_root); + } else { + if (fragment_entry->next != nullptr) { + // In this case a fragment in the middle finished + std::unique_ptr next = std::move(fragment_entry->next); + next->prev = prev; + // This next line will cause fragment_entry to be deleted + prev->next = std::move(next); + } else { + // In this case a fragment at the end finished + // This next line will cause fragment_entry to be deleted + prev->next = nullptr; + } + } + num_fragments_processed_++; + if (num_fragments_processed_ == total_num_fragments_) { + std::cout << " Completion via fragment end" << std::endl; + completion_cb_(); + return; + } + } + TryAndLaunchTasksUnlocked(); + } + + int queue_limit_; + int run_limit_; + util::ThrottledAsyncTaskScheduler* inspection_throttle_; + std::function completion_cb_; + + int num_queued_scan_tasks_ = 0; + int num_scan_tasks_running_ = 0; + int batches_completed_ = 0; + std::unique_ptr root_; + std::mutex mutex_; + int num_fragments_processed_ = 0; + int total_num_fragments_ = -1; +}; + Result> OutputSchemaFromOptions(const ScanV2Options& options) { return FieldPath::GetAll(*options.dataset->schema(), options.columns); } @@ -80,40 +331,45 @@ Future>> GetFragments( /// \brief A node that scans a dataset /// -/// The scan node has three groups of io-tasks and one task. +/// The scan node has three stages. /// -/// The first io-task (listing) fetches the fragments from the dataset. This may be a +/// The first stage (listing) fetches the fragments from the dataset. This may be a /// simple iteration of paths or, if the dataset is described with wildcards, this may -/// involve I/O for listing and walking directory paths. There is one listing io-task -/// per dataset. +/// involve I/O for listing and walking directory paths. There is only one listing task. /// -/// If a fragment has a guarantee we may use that expression to reduce the columns that -/// we need to load from disk. For example, if the guarantee is x==7 then we don't need -/// to load the column x from disk and can instead populate x with the scalar 7. If the -/// fragment on disk actually had a column x, and the value was not 7, then we will prefer -/// the guarantee in this invalid case. +/// Ths next step is to inspect the fragment. At a minimum we need to know the names of +/// the columns in the fragment so that we can perform evolution. This often involves +/// reading file metadata or the first block of the file (e.g. CSV / JSON). There is one +/// inspect task per fragment. +/// +/// Once the inspect is done we can issue scan tasks. The number of scan tasks created +/// will depend on the format and the file structure. For example, CSV creates 1 scan +/// task per file. Parquet creates one scan task per row group. /// -/// Ths next step is to fetch the metadata for the fragment. For some formats (e.g. -/// CSV) this may be quite simple (get the size of the file). For other formats (e.g. -/// parquet) this is more involved and requires reading data. There is one metadata -/// io-task per fragment. The metadata io-task creates an AsyncGenerator -/// from the fragment. +/// The creation of scan tasks is a partially sequenced operation. We can start +/// inspecting fragments in parallel. However, we cannot start scanning a fragment until +/// we know how many batches will come before that fragment. This allows us to assign a +/// sequence number to scanned batches. /// -/// Once the metadata io-task is done we can issue read io-tasks. Each read io-task -/// requests a single batch of data from the disk by pulling the next Future from the -/// generator. +/// Each scan task is then broken up into a series of batch scans which scan a single +/// batch from the disk. For example, in parquet, we might have a very large row group. +/// That single row group will have one scan task. That scan task might generate hundreds +/// of batch scans. /// -/// Finally, when the future is fulfilled, we issue a pipeline task to drive the batch -/// through the pipeline. +/// Finally, when the batch scan is complete, we issue a pipeline task to drive the batch +/// through the plan. /// -/// Most of these tasks are io-tasks. They take very few CPU resources and they run on -/// the I/O thread pool. These io-tasks are invisible to the exec plan and so we need -/// to do some custom scheduling. We limit how many fragments we read from at any one -/// time. This is referred to as "fragment readahead". +/// Most of these tasks are I/O tasks. They take very few CPU resources and they run on +/// the I/O thread pool. /// -/// Within a fragment there is usually also some amount of "row readahead". This row -/// readahead is handled by the fragment (and not the scanner) because the exact details -/// of how it is performed depend on the underlying format. +/// In order to manage the disk load and our running memory requirements we limit the +/// number of scan tasks that run at any one time. +/// +/// If a fragment has a guarantee we may use that expression to reduce the columns that +/// we need to load from disk. For example, if the guarantee is x==7 then we don't need +/// to load the column x from disk and can instead populate x with the scalar 7. If the +/// fragment on disk actually had a column x, and the value was not 7, then we will prefer +/// the guarantee in this invalid case. /// /// When a scan node is aborted (StopProducing) we send a cancel signal to any active /// fragments. On destruction we continue consuming the fragments until they complete @@ -125,7 +381,7 @@ class ScanNode : public acero::ExecNode, public acero::TracedNode { std::shared_ptr output_schema) : acero::ExecNode(plan, {}, {}, std::move(output_schema)), acero::TracedNode(this), - options_(options) {} + options_(std::move(options)) {} static Result NormalizeAndValidate(const ScanV2Options& options, compute::ExecContext* ctx) { @@ -134,14 +390,9 @@ class ScanNode : public acero::ExecNode, public acero::TracedNode { return Status::Invalid("Scan options must include a dataset"); } - if (options.fragment_readahead < 0) { - return Status::Invalid( - "Fragment readahead may not be less than 0. Set to 0 to disable readahead"); - } - - if (options.target_bytes_readahead < 0) { + if (options.scan_task_readahead < 0) { return Status::Invalid( - "Batch readahead may not be less than 0. Set to 0 to disable readahead"); + "Scan task readahead may not be less than 0. Set to 0 to disable readahead"); } if (!normalized.filter.is_valid()) { @@ -199,51 +450,67 @@ class ScanNode : public acero::ExecNode, public acero::TracedNode { Datum value; }; - struct ScanState { - std::mutex mutex; - std::shared_ptr fragment_scanner; - std::unique_ptr fragment_evolution; - std::vector known_values; - FragmentScanRequest scan_request; - }; - - struct ScanBatchTask : public util::AsyncTaskScheduler::Task { - ScanBatchTask(ScanNode* node, ScanState* scan_state, int batch_index) - : node_(node), scan_(scan_state), batch_index_(batch_index) { - int64_t cost = scan_state->fragment_scanner->EstimatedDataBytes(batch_index_); - // It's possible, though probably a bad idea, for a single batch of a fragment - // to be larger than 2GiB. In that case, it doesn't matter much if we - // underestimate because the largest the throttle can be is 2GiB and thus we will - // be in "one batch at a time" mode anyways which is the best we can do in this - // case. - cost_ = static_cast( - std::min(cost, static_cast(std::numeric_limits::max()))); - name_ = "ScanNode::ScanBatch::" + ::arrow::internal::ToChars(batch_index_); - } - - Result> operator()() override { - // Prevent concurrent calls to ScanBatch which might not be thread safe - std::lock_guard lk(scan_->mutex); - return scan_->fragment_scanner->ScanBatch(batch_index_) - .Then([this](const std::shared_ptr& batch) { - return HandleBatch(batch); - }); + // The staging area and the fragment scanner don't want to know any details about + // evolution Those details are encapsulated here. The inspection task calculates + // exactly how to evolve outgoing batches and creates a template that is used by all + // scan tasks in the fragment + class ScanTaskLauncherImpl : public ScanTaskLauncher { + public: + ScanTaskLauncherImpl(ScanNode* node, + std::unique_ptr fragment_evolution, + std::vector known_values, + FragmentScanRequest scan_request) + : node_(node), + fragment_evolution_(std::move(fragment_evolution)), + known_values_(std::move(known_values)), + scan_request_(std::move(scan_request)) {} + + void LaunchTask(FragmentScanner* scanner, int scan_task_number, int first_batch_index, + std::function task_complete_cb) override { + AsyncGenerator> batch_gen = + scanner->RunScanTask(scan_task_number); + auto batch_count = std::make_shared(0); + node_->plan_->query_context() + ->async_scheduler() + ->AddAsyncGenerator>( + std::move(batch_gen), + [this, scan_task_number, batch_counter = batch_count.get()]( + const std::shared_ptr& batch) { + std::cout << " ScanTask(" << scan_task_number << "," << (*batch_counter) + << ")" << std::endl; + (*batch_counter)++; + return HandleBatch(batch); + }, + "ScanNode::ScanBatch::Next", + [this, scan_task_number, task_complete_cb = std::move(task_complete_cb), + batch_count = std::move(batch_count)]() { + node_->plan_->query_context()->ScheduleTask( + [scan_task_number, task_complete_cb, batch_count] { + std::cout << " Scan task complete(" << scan_task_number << ")" + << std::endl; + task_complete_cb(*batch_count); + return Status::OK(); + }, + "ScanTaskWrapUp"); + return Status::OK(); + }); } - std::string_view name() const override { return name_; } + const FragmentScanRequest& scan_request() const { return scan_request_; } + private: compute::ExecBatch AddKnownValues(compute::ExecBatch batch) { - if (scan_->known_values.empty()) { + if (known_values_.empty()) { return batch; } std::vector with_known_values; int num_combined_cols = - static_cast(batch.values.size() + scan_->known_values.size()); + static_cast(batch.values.size() + known_values_.size()); with_known_values.reserve(num_combined_cols); - auto known_values_itr = scan_->known_values.cbegin(); + auto known_values_itr = known_values_.cbegin(); auto batch_itr = batch.values.begin(); for (int i = 0; i < num_combined_cols; i++) { - if (known_values_itr != scan_->known_values.end() && + if (known_values_itr != known_values_.end() && static_cast(known_values_itr->index) == i) { with_known_values.push_back(known_values_itr->value); known_values_itr++; @@ -258,38 +525,39 @@ class ScanNode : public acero::ExecNode, public acero::TracedNode { Status HandleBatch(const std::shared_ptr& batch) { ARROW_ASSIGN_OR_RAISE( compute::ExecBatch evolved_batch, - scan_->fragment_evolution->EvolveBatch( - batch, node_->options_.columns, *scan_->scan_request.fragment_selection)); + fragment_evolution_->EvolveBatch(batch, node_->options_.columns, + *scan_request_.fragment_selection)); compute::ExecBatch with_known_values = AddKnownValues(std::move(evolved_batch)); node_->plan_->query_context()->ScheduleTask( [node = node_, output_batch = std::move(with_known_values)] { + node->batch_counter_++; return node->output_->InputReceived(node, output_batch); }, "ScanNode::ProcessMorsel"); return Status::OK(); } - int cost() const override { return cost_; } - ScanNode* node_; - ScanState* scan_; - int batch_index_; - int cost_; - std::string name_; + const std::unique_ptr fragment_evolution_; + const std::vector known_values_; + const FragmentScanRequest scan_request_; }; - struct ListFragmentTask : util::AsyncTaskScheduler::Task { - ListFragmentTask(ScanNode* node, std::shared_ptr fragment) - : node(node), fragment(std::move(fragment)) { - name_ = "ScanNode::ListFragment::" + this->fragment->ToString(); + struct InspectFragmentTask : util::AsyncTaskScheduler::Task { + InspectFragmentTask(ScanNode* node, std::shared_ptr fragment, + int fragment_index) + : node_(node), fragment_(std::move(fragment)), fragment_index_(fragment_index) { + name_ = "ScanNode::InspectFragment::" + fragment_->ToString(); } Result> operator()() override { - return fragment - ->InspectFragment(node->options_.format_options, - node->plan_->query_context()->exec_context()) + std::cout << "START: Inspecting fragment: " << fragment_->ToString() << std::endl; + return fragment_ + ->InspectFragment(node_->options_.format_options, + node_->plan_->query_context()->exec_context(), + node_->options_.cache_fragment_inspection) .Then([this](const std::shared_ptr& inspected_fragment) { - return BeginScan(inspected_fragment); + return OnInspectionComplete(inspected_fragment); }); } @@ -306,10 +574,10 @@ class ScanNode : public acero::ExecNode, public acero::TracedNode { const compute::Expression& guarantee) { ARROW_ASSIGN_OR_RAISE( compute::KnownFieldValues part_values, - compute::ExtractKnownFieldValues(fragment->partition_expression())); + compute::ExtractKnownFieldValues(fragment_->partition_expression())); ExtractedKnownValues extracted; - for (std::size_t i = 0; i < node->options_.columns.size(); i++) { - const auto& field_path = node->options_.columns[i]; + for (std::size_t i = 0; i < node_->options_.columns.size(); i++) { + const auto& field_path = node_->options_.columns[i]; FieldRef field_ref(field_path); auto existing = part_values.map.find(FieldRef(field_path)); if (existing == part_values.map.end()) { @@ -317,7 +585,7 @@ class ScanNode : public acero::ExecNode, public acero::TracedNode { extracted.remaining_columns.push_back(field_path); } else { ARROW_ASSIGN_OR_RAISE(const std::shared_ptr& field, - field_path.Get(*node->options_.dataset->schema())); + field_path.Get(*node_->options_.dataset->schema())); Result maybe_casted = compute::Cast(existing->second, field->type()); if (!maybe_casted.ok()) { // TODO(weston) In theory this should be preventable. The dataset and @@ -336,117 +604,109 @@ class ScanNode : public acero::ExecNode, public acero::TracedNode { return std::move(extracted); } - Future<> BeginScan(const std::shared_ptr& inspected_fragment) { + Future<> OnInspectionComplete( + const std::shared_ptr& inspected_fragment) { // Based on the fragment's guarantee we may not need to retrieve all the columns - compute::Expression fragment_filter = node->options_.filter; + compute::Expression fragment_filter = node_->options_.filter; ARROW_ASSIGN_OR_RAISE( compute::Expression filter_minus_part, compute::SimplifyWithGuarantee(std::move(fragment_filter), - fragment->partition_expression())); + fragment_->partition_expression())); ARROW_ASSIGN_OR_RAISE( ExtractedKnownValues extracted, - ExtractKnownValuesFromGuarantee(fragment->partition_expression())); + ExtractKnownValuesFromGuarantee(fragment_->partition_expression())); // Now that we have an inspected fragment we need to use the dataset's evolution // strategy to figure out how to scan it - scan_state->fragment_evolution = - node->options_.dataset->evolution_strategy()->GetStrategy( - *node->options_.dataset, *fragment, *inspected_fragment); - ARROW_RETURN_NOT_OK(InitFragmentScanRequest(extracted.remaining_columns, - filter_minus_part, - std::move(extracted.known_values))); - return fragment - ->BeginScan(scan_state->scan_request, *inspected_fragment, - node->options_.format_options, - node->plan_->query_context()->exec_context()) - .Then([this](const std::shared_ptr& fragment_scanner) { - return AddScanTasks(fragment_scanner); + std::unique_ptr fragment_evolution = + node_->options_.dataset->evolution_strategy()->GetStrategy( + *node_->options_.dataset, *fragment_, *inspected_fragment); + ARROW_ASSIGN_OR_RAISE( + std::unique_ptr task_launcher, + CreateTaskLauncher(std::move(fragment_evolution), extracted.remaining_columns, + filter_minus_part, std::move(extracted.known_values))); + + std::cout << " BeginScan: " << fragment_->ToString() << std::endl; + return fragment_ + ->BeginScan(task_launcher->scan_request(), *inspected_fragment, + node_->plan_->query_context()->exec_context()) + .Then([this, task_launcher = std::move(task_launcher)]( + const std::shared_ptr& fragment_scanner) mutable { + node_->staging_area_->InsertInspectedFragment( + fragment_scanner, fragment_index_, std::move(task_launcher)); }); } - Future<> AddScanTasks(const std::shared_ptr& fragment_scanner) { - scan_state->fragment_scanner = fragment_scanner; - ScanState* state_view = scan_state.get(); - Future<> list_and_scan_done = Future<>::Make(); - // Finish callback keeps the scan state alive until all scan tasks done - struct StateHolder { - Status operator()() { - list_and_scan_done.MarkFinished(); - return Status::OK(); - } - Future<> list_and_scan_done; - std::unique_ptr scan_state; - }; - - std::unique_ptr scan_tasks = util::AsyncTaskGroup::Make( - node->batches_throttle_.get(), - StateHolder{list_and_scan_done, std::move(scan_state)}); - for (int i = 0; i < fragment_scanner->NumBatches(); i++) { - node->num_batches_.fetch_add(1); - scan_tasks->AddTask(std::make_unique(node, state_view, i)); - } - return Status::OK(); - // The "list fragments" task doesn't actually end until the fragments are - // all scanned. This allows us to enforce fragment readahead. - return list_and_scan_done; - } - // Take the dataset options, and the fragment evolution, and figure out exactly how // we should scan the fragment itself. - Status InitFragmentScanRequest(const std::vector& desired_columns, - const compute::Expression& filter, - std::vector known_values) { - ARROW_ASSIGN_OR_RAISE( - scan_state->scan_request.fragment_selection, - scan_state->fragment_evolution->DevolveSelection(desired_columns)); - ARROW_ASSIGN_OR_RAISE( - compute::Expression devolution_guarantee, - scan_state->fragment_evolution->GetGuarantee(desired_columns)); + Result> CreateTaskLauncher( + std::unique_ptr fragment_evolution, + const std::vector& desired_columns, const compute::Expression& filter, + std::vector known_values) { + FragmentScanRequest scan_request; + ARROW_ASSIGN_OR_RAISE(scan_request.fragment_selection, + fragment_evolution->DevolveSelection(desired_columns)); + ARROW_ASSIGN_OR_RAISE(compute::Expression devolution_guarantee, + fragment_evolution->GetGuarantee(desired_columns)); ARROW_ASSIGN_OR_RAISE(compute::Expression simplified_filter, compute::SimplifyWithGuarantee(filter, devolution_guarantee)); - ARROW_ASSIGN_OR_RAISE( - scan_state->scan_request.filter, - scan_state->fragment_evolution->DevolveFilter(std::move(simplified_filter))); - scan_state->scan_request.format_scan_options = node->options_.format_options; - scan_state->known_values = std::move(known_values); - return Status::OK(); + ARROW_ASSIGN_OR_RAISE(scan_request.filter, fragment_evolution->DevolveFilter( + std::move(simplified_filter))); + scan_request.format_scan_options = node_->options_.format_options; + auto task_launcher = std::make_unique( + node_, std::move(fragment_evolution), std::move(known_values), + std::move(scan_request)); + return task_launcher; } - ScanNode* node; - std::shared_ptr fragment; - std::unique_ptr scan_state = std::make_unique(); + ScanNode* node_; + std::shared_ptr fragment_; + int fragment_index_; std::string name_; }; - void ScanFragments(const AsyncGenerator>& frag_gen) { - std::shared_ptr fragment_tasks = - util::MakeThrottledAsyncTaskGroup( - plan_->query_context()->async_scheduler(), options_.fragment_readahead + 1, - /*queue=*/nullptr, - [this]() { return output_->InputFinished(this, num_batches_.load()); }); - fragment_tasks->AddAsyncGenerator>( - std::move(frag_gen), - [this, fragment_tasks = - std::move(fragment_tasks)](const std::shared_ptr& fragment) { - fragment_tasks->AddTask(std::make_unique(this, fragment)); - return Status::OK(); - }, - "ScanNode::ListDataset::Next"); + void InspectFragments(const AsyncGenerator>& frag_gen) { + plan_->query_context() + ->async_scheduler() + ->AddAsyncGenerator>( + std::move(frag_gen), + [this](const std::shared_ptr& fragment) { + std::cout << " Listed fragment: " << fragment->ToString() << std::endl; + inspection_throttle_->AddTask(std::make_unique( + this, fragment, fragment_index_++)); + return Status::OK(); + }, + "ScanNode::ListDataset::Next", + [this] { + staging_area_->FinishedInsertingFragments(fragment_index_); + return Status::OK(); + }); } Status StartProducing() override { NoteStartProducing(ToStringExtra()); - batches_throttle_ = util::ThrottledAsyncTaskScheduler::Make( - plan_->query_context()->async_scheduler(), options_.target_bytes_readahead + 1); + inspection_throttle_ = util::ThrottledAsyncTaskScheduler::Make( + plan_->query_context()->async_scheduler(), kNumConcurrentInspections); + auto completion = [this] { + plan_->query_context()->ScheduleTask( + [this] { return output_->InputFinished(this, batch_counter_.load()); }, + "ScanNode::Finished"); + }; + // For ease of use we treat readahead=1 as "only scan one thing at a time" + int scan_task_readahead = std::max(options_.scan_task_readahead, 1); + staging_area_ = std::make_unique( + kLimitQueuedScanTasks, scan_task_readahead, inspection_throttle_.get(), + std::move(completion)); plan_->query_context()->async_scheduler()->AddSimpleTask( [this] { + std::cout << "START: ListFragments::GetFragments" << std::endl; return GetFragments(options_.dataset.get(), options_.filter) .Then([this](const AsyncGenerator>& frag_gen) { - ScanFragments(frag_gen); + InspectFragments(frag_gen); }); }, - "ScanNode::ListDataset::GetFragments"sv); + "ScanNode::StartListing"sv); return Status::OK(); } @@ -462,8 +722,10 @@ class ScanNode : public acero::ExecNode, public acero::TracedNode { private: ScanV2Options options_; - std::atomic num_batches_{0}; - std::shared_ptr batches_throttle_; + std::shared_ptr inspection_throttle_; + std::unique_ptr staging_area_; + int fragment_index_ = 0; + std::atomic batch_counter_{0}; }; } // namespace diff --git a/cpp/src/arrow/dataset/scanner.h b/cpp/src/arrow/dataset/scanner.h index 5479a0d9db404..9ecde2fc12aba 100644 --- a/cpp/src/arrow/dataset/scanner.h +++ b/cpp/src/arrow/dataset/scanner.h @@ -54,6 +54,11 @@ constexpr int64_t kDefaultBatchSize = 1 << 17; // 128Ki rows // This will yield 64 batches ~ 8Mi rows constexpr int32_t kDefaultBatchReadahead = 16; constexpr int32_t kDefaultFragmentReadahead = 4; + +// Note, the above properties are specific to the legacy scanner. These following +// properties control the new scanner. This configuration should lead to max of ~512MiB +// of scanner buffer while still allowing 16 parallel read streams. +constexpr int32_t kDefaultScanTaskReadahead = 16; constexpr int32_t kDefaultBytesReadahead = 1 << 25; // 32MiB /// Scan-specific options, which can be changed between scans of the same dataset. @@ -219,39 +224,59 @@ struct ARROW_DS_EXPORT ScanV2Options : public acero::ExecNodeOptions { /// }) std::vector columns; - /// \brief Target number of bytes to read ahead in a fragment - /// - /// This limit involves some amount of estimation. Formats typically only know - /// batch boundaries in terms of rows (not decoded bytes) and so an estimation - /// must be done to guess the average row size. Other formats like CSV and JSON - /// must make even more generalized guesses. - /// - /// This is a best-effort guide. Some formats may need to read ahead further, - /// for example, if scanning a parquet file that has batches with 100MiB of data - /// then the actual readahead will be at least 100MiB - /// - /// Set to 0 to disable readhead. When disabled, the scanner will read the - /// dataset one batch at a time - /// - /// This limit applies across all fragments. If the limit is 32MiB and the - /// fragment readahead allows for 20 fragments to be read at once then the - /// total readahead will still be 32MiB and NOT 20 * 32MiB. - int32_t target_bytes_readahead = kDefaultBytesReadahead; - - /// \brief Number of fragments to read ahead + /// \brief How many concurrent streams should the scanner read /// - /// Higher readahead will potentially lead to more efficient I/O but will lead + /// Higher readahead could potentially lead to more efficient I/O but will lead /// to the scan operation using more RAM. The default is fairly conservative /// and designed for fast local disks (or slow local spinning disks which cannot /// handle much parallelism anyways). When using a highly parallel remote filesystem /// you will likely want to increase these values. /// - /// Set to 0 to disable fragment readahead. When disabled the dataset will be scanned - /// one fragment at a time. - int32_t fragment_readahead = kDefaultFragmentReadahead; + /// Set to 0 to disable readahead. When disabled the dataset will be scanned + /// one stream at a time. + /// + /// A stream (or scan task) is any independently scannable unit. For example, an + /// entire JSON file is a single stream. We scan the file from start to finish and + /// cannot start scanning in the middle. A parquet file has one stream per row group. + /// This is because we can scan the row groups in parallel. + int32_t scan_task_readahead = kDefaultScanTaskReadahead; /// \brief Options specific to the file format const FragmentScanOptions* format_options = NULLPTR; + /// \brief How many rows to skip + /// + /// This skips rows according to the implicit order of the dataset + /// + /// Not all file formats support skipping. For example, when scanning CSV we + /// do not know ahead of time how many rows are in a batch. Since we are typically + /// scanning in parallel this means we cannot reasonable enforce a skip. + /// + /// Also note that this skip happens before any filtering of the data. This means + /// that you typically do not want to do skipping if you are also filtering. + /// + /// If you want to skip after filtering or you want to skip and your file format does + /// not support it then you should use a \see arrow::compute::FetchNode + int64_t rows_to_skip = 0; + + /// \brief How many rows to read + /// + /// The default (nullopt) will read all rows + /// + /// This will read the first `rows_to_read` rows (after any skip) according to + /// the implicit order of the dataset. + /// + /// Similar to `rows_to_skip` this is not supported by all file formats and does + /// not typically make sense when you are also using a filter. In those cases you + /// should apply your limit after the scan using \see arrow::compute::FetchNode + std::optional rows_to_read = std::nullopt; + + /// \brief Should the scan node cache fragment inspection information + /// + /// If this is true then future scans of the same fragment instance may start slightly + /// faster. However, this information could take up a large amount of RAM if there are + /// many fragments. + bool cache_fragment_inspection = false; + /// \brief Utility method to get a selection representing all columns in a dataset static std::vector AllColumns(const Schema& dataset_schema); diff --git a/cpp/src/arrow/dataset/scanner_test.cc b/cpp/src/arrow/dataset/scanner_test.cc index cde3a725c4663..4a1fa01fbfb9a 100644 --- a/cpp/src/arrow/dataset/scanner_test.cc +++ b/cpp/src/arrow/dataset/scanner_test.cc @@ -42,6 +42,7 @@ #include "arrow/testing/matchers.h" #include "arrow/testing/util.h" #include "arrow/util/byte_size.h" +#include "arrow/util/iterator.h" #include "arrow/util/range.h" #include "arrow/util/thread_pool.h" #include "arrow/util/vector.h" @@ -163,6 +164,8 @@ TEST(BasicEvolution, ReorderedColumns) { struct MockScanTask { explicit MockScanTask(std::shared_ptr batch) : batch(std::move(batch)) {} + void Finish() { batch_future.MarkFinished(batch); } + std::shared_ptr batch; Future> batch_future = Future>::Make(); @@ -188,14 +191,26 @@ struct MockFragmentScanner : public FragmentScanner { : scan_tasks_(std::move(scan_tasks)), has_started_(scan_tasks_.size(), false) {} // ### FragmentScanner API ### - Future> ScanBatch(int batch_number) override { - has_started_[batch_number] = true; - return scan_tasks_[batch_number].batch_future; + AsyncGenerator> RunScanTask( + int scan_task_number) override { + auto batch_number = std::make_shared(0); + return [this, batch_number] { + if (*batch_number == static_cast(scan_tasks_.size())) { + return AsyncGeneratorEnd>(); + } + has_started_[*batch_number] = true; + Future> batch = + scan_tasks_[*batch_number].batch_future; + (*batch_number)++; + return batch; + }; } - int64_t EstimatedDataBytes(int batch_number) override { - return util::TotalBufferSize(*scan_tasks_[batch_number].batch); + + int NumScanTasks() override { return 1; } + + int NumBatchesInScanTask(int task_number) override { + return static_cast(scan_tasks_.size()); } - int NumBatches() override { return static_cast(scan_tasks_.size()); } // ### Unit Test API ### void DeliverBatches(bool slow, const std::vector& to_deliver) { @@ -224,6 +239,8 @@ struct MockFragmentScanner : public FragmentScanner { return scan_tasks_[batch_number].batch_future.is_finished(); } + int num_batches() { return static_cast(scan_tasks_.size()); } + std::vector scan_tasks_; std::vector has_started_; }; @@ -245,15 +262,14 @@ struct MockFragment : public Fragment { }; Future> InspectFragment( - const FragmentScanOptions* format_options, - compute::ExecContext* exec_context) override { + const FragmentScanOptions* format_options, compute::ExecContext* exec_context, + bool should_cache) override { has_inspected_ = true; return inspected_future_; } Future> BeginScan( const FragmentScanRequest& request, const InspectedFragment& inspected_fragment, - const FragmentScanOptions* format_options, compute::ExecContext* exec_context) override { has_started_ = true; seen_request_ = request; @@ -582,8 +598,11 @@ INSTANTIATE_TEST_SUITE_P(BasicNewScannerTests, TestScannerBase, }); void CheckScannerBackpressure(std::shared_ptr dataset, ScanV2Options options, - int maxConcurrentFragments, int maxConcurrentBatches, + int maxConcurrentScanTasks, ::arrow::internal::ThreadPool* thread_pool) { + // These are hard-coded and not configurable + constexpr int kMaxInspectionsReadahead = 4; + // constexpr int kMaxQueuedScanTasks = 4; // Start scanning acero::Declaration scan_decl = acero::Declaration("scan2", std::move(options)); Future batches_fut = @@ -598,15 +617,21 @@ void CheckScannerBackpressure(std::shared_ptr dataset, ScanV2Option } return num_inspected; }; + // Since inspection is not finishing we shouldn't get more than kMaxInspectionReadahead + // inspections running at once BusyWait(10, [&] { - return get_num_inspected() == static_cast(maxConcurrentFragments); + return get_num_inspected() >= static_cast(kMaxInspectionsReadahead); }); + std::cout << "### Wait done. Ensuring inspections readahead limit reached ###" + << std::endl; SleepABit(); - ASSERT_EQ(get_num_inspected(), static_cast(maxConcurrentFragments)); + ASSERT_EQ(get_num_inspected(), static_cast(kMaxInspectionsReadahead)); + std::cout << "### Assert passed. Finishing inspections and starting scan ###" + << std::endl; int total_batches = 0; for (const auto& frag : dataset->fragments_) { - total_batches += frag->fragment_scanner_->NumBatches(); + total_batches += frag->fragment_scanner_->num_batches(); frag->FinishInspection(); frag->FinishScanBegin(); } @@ -617,7 +642,7 @@ void CheckScannerBackpressure(std::shared_ptr dataset, ScanV2Option thread_pool->WaitForIdle(); int batches_started = 0; for (const auto& frag : dataset->fragments_) { - for (int i = 0; i < frag->fragment_scanner_->NumBatches(); i++) { + for (int i = 0; i < frag->fragment_scanner_->num_batches(); i++) { if (frag->HasBatchStarted(i)) { batches_started++; if (next_task_to_deliver == nullptr && !frag->HasBatchDelivered(i)) { @@ -626,7 +651,7 @@ void CheckScannerBackpressure(std::shared_ptr dataset, ScanV2Option } } } - ASSERT_LE(batches_started - batches_scanned, maxConcurrentBatches) + ASSERT_LE(batches_started - batches_scanned, maxConcurrentScanTasks) << " too many scan tasks were allowed to run"; ASSERT_NE(next_task_to_deliver, nullptr); next_task_to_deliver->batch_future.MarkFinished(next_task_to_deliver->batch); @@ -635,7 +660,7 @@ void CheckScannerBackpressure(std::shared_ptr dataset, ScanV2Option } TEST(TestNewScanner, Backpressure) { - constexpr int kNumFragments = 4; + constexpr int kNumFragments = 20; constexpr int kNumBatchesPerFragment = 4; internal::Initialize(); std::shared_ptr test_dataset = @@ -646,22 +671,62 @@ TEST(TestNewScanner, Backpressure) { // No readahead options.dataset = test_dataset; options.columns = ScanV2Options::AllColumns(*test_dataset->schema()); - options.fragment_readahead = 0; - options.target_bytes_readahead = 0; - CheckScannerBackpressure(test_dataset, options, 1, 1, + options.scan_task_readahead = 0; + CheckScannerBackpressure(test_dataset, options, /*max_scan_tasks=*/1, ::arrow::internal::GetCpuThreadPool()); // Some readahead test_dataset = MakeTestDataset(kNumFragments, kNumBatchesPerFragment); options = ScanV2Options(test_dataset); options.columns = ScanV2Options::AllColumns(*test_dataset->schema()); - options.fragment_readahead = 4; - // each batch should be 14Ki so 50Ki readahead should yield 3-at-a-time - options.target_bytes_readahead = 50 * kRowsPerTestBatch; - CheckScannerBackpressure(test_dataset, options, 4, 3, + options.scan_task_readahead = 4; + CheckScannerBackpressure(test_dataset, options, /*max_scan_tasks=*/4, ::arrow::internal::GetCpuThreadPool()); } +// Test a fragment completing when a previous fragment is still running +TEST(TestNewScanner, OutOfOrderFragmentCompletion) { + constexpr int kNumFragments = 3; + constexpr int kNumBatchesPerFragment = 1; + + internal::Initialize(); + std::shared_ptr test_dataset = + MakeTestDataset(kNumFragments, kNumBatchesPerFragment); + + ScanV2Options options(test_dataset); + options.columns = ScanV2Options::AllColumns(*test_dataset->schema()); + + // Begin scan + acero::Declaration scan_decl = acero::Declaration("scan2", std::move(options)); + Future batches_fut = + acero::DeclarationToBatchesAsync(std::move(scan_decl)); + + // Start scanning on all fragments + for (int i = 0; i < kNumFragments; i++) { + test_dataset->fragments_[i]->FinishInspection(); + test_dataset->fragments_[i]->FinishScanBegin(); + } + + // Let scan tasks get started + SleepABit(); + // A fragment in the middle finishes + test_dataset->fragments_[1]->fragment_scanner_->scan_tasks_[0].Finish(); + + SleepABit(); + + // A fragment at the end finishes + test_dataset->fragments_[2]->fragment_scanner_->scan_tasks_[0].Finish(); + + SleepABit(); + + // Now the first fragment finishes + test_dataset->fragments_[0]->fragment_scanner_->scan_tasks_[0].Finish(); + + // The scan should finish cleanly + ASSERT_FINISHES_OK_AND_ASSIGN(RecordBatchVector batches, batches_fut); + ASSERT_EQ(3, batches.size()); +} + TEST(TestNewScanner, NestedRead) { // This tests the case where the file format does not support // handling nested reads (e.g. JSON) and so the scanner must diff --git a/cpp/src/arrow/dataset/test_util_internal.h b/cpp/src/arrow/dataset/test_util_internal.h index 51d39d532c82c..cb9e14ec97d2d 100644 --- a/cpp/src/arrow/dataset/test_util_internal.h +++ b/cpp/src/arrow/dataset/test_util_internal.h @@ -36,6 +36,8 @@ #include "arrow/compute/exec.h" #include "arrow/compute/expression.h" #include "arrow/compute/kernel.h" +#include "arrow/compute/type_fwd.h" +#include "arrow/dataset/dataset.h" #include "arrow/dataset/dataset_internal.h" #include "arrow/dataset/discovery.h" #include "arrow/dataset/file_base.h" @@ -44,6 +46,7 @@ #include "arrow/filesystem/path_util.h" #include "arrow/filesystem/test_util.h" #include "arrow/record_batch.h" +#include "arrow/status.h" #include "arrow/table.h" #include "arrow/testing/future_util.h" #include "arrow/testing/generator.h" @@ -58,6 +61,7 @@ #include "arrow/util/pcg_random.h" #include "arrow/util/thread_pool.h" #include "arrow/util/vector.h" +#include "gmock/gmock.h" namespace arrow { @@ -1021,6 +1025,11 @@ class FileFormatScanMixin : public FileFormatFixtureMixin, using FileFormatFixtureMixin::opts_; }; +class InvalidFragmentScanOptions : public FragmentScanOptions { + public: + std::string type_name() const override { return "invalid"; } +}; + template class FileFormatFixtureMixinV2 : public ::testing::Test { public: @@ -1157,6 +1166,18 @@ class FileFormatFixtureMixinV2 : public ::testing::Test { AssertSchemaEqual(*actual, *reader->schema(), /*check_metadata=*/false); } + void TestInvalidFormatScanOptions() { + auto reader = this->GetRandomData(schema({field("f64", float64())})); + auto source = this->MakeBufferSource(reader.get()); + + InvalidFragmentScanOptions invalid_options; + auto fragment = this->MakeFragment(*source); + ASSERT_THAT( + fragment->InspectFragmentImpl(&invalid_options, compute::default_exec_context()), + Finishes(Raises(StatusCode::Invalid, + testing::HasSubstr("scan options of type invalid")))); + } + void TestIsSupported() { auto reader = GetRandomData(schema({field("f64", float64())})); auto source = MakeBufferSource(reader.get()); @@ -1286,8 +1307,7 @@ class FileFormatScanNodeMixin : public FileFormatFixtureMixinV2, RecordBatchIterator Batches(const std::shared_ptr& fragment, bool use_readahead = true) { if (!use_readahead) { - opts_->target_bytes_readahead = 0; - opts_->fragment_readahead = 0; + opts_->scan_task_readahead = 0; } EXPECT_OK_AND_ASSIGN(auto reader, this->Scan(fragment)); struct ReaderIterator { @@ -1297,7 +1317,13 @@ class FileFormatScanNodeMixin : public FileFormatFixtureMixinV2, return RecordBatchIterator(ReaderIterator{std::move(reader)}); } - // Shared test cases + // Basic scan test + // + // We generate some random data and encode it in a buffer as parquet + // The dataset schema matches the data schema + // We create one fragment from the buffer + // We scan all columns and all rows + // We ensure we get the correct # of rows back void TestScan() { // Basic test to make sure we can scan data auto random_data = GetRandomData(schema({field("f64", float64())})); @@ -1315,11 +1341,13 @@ class FileFormatScanNodeMixin : public FileFormatFixtureMixinV2, ASSERT_EQ(row_count, GetParam().expected_rows()); } - // TestScanBatchSize is no longer relevant because batch size is an internal concern. - // Consumers should only really care about batch sizing at the sink. - - // Ensure file formats only return columns needed to fulfill filter/projection - void TestScanProjected() { + // Ensure file formats only return columns that are asked for in `columns` + // We create test data with 4 columns + // We ask for only one of them (f64) + // We set a row filter that relies on a different column (i32) + // This row filter matches all rows + // We expect to get back all rows and only the one column we asked for + void TestScanSomeColumns() { auto f32 = field("f32", float32()); auto f64 = field("f64", float64()); auto i32 = field("i32", int32()); @@ -1328,8 +1356,9 @@ class FileFormatScanNodeMixin : public FileFormatFixtureMixinV2, this->SetScanProjectionRefs({"f64"}); this->SetScanFilter(equal(field_ref("i32"), literal(0))); - // We expect f64 since it is asked for and i32 since it is needed for the filter - auto expected_schema = schema({f64, i32}); + // We expect f64 since it is asked for + // Even though i32 is asked for in the filter, it should not be returned + auto expected_schema = schema({f64}); auto reader = this->GetRandomData(dataset_schema_); auto source = this->MakeBufferSource(reader.get()); @@ -1338,7 +1367,7 @@ class FileFormatScanNodeMixin : public FileFormatFixtureMixinV2, int64_t row_count = 0; ASSERT_OK_AND_ASSIGN(std::unique_ptr scanner, - this->Scan(fragment)); + this->Scan(fragment, /*add_filter_fields=*/false)); for (auto maybe_batch : *scanner) { ASSERT_OK_AND_ASSIGN(auto batch, maybe_batch); row_count += batch->num_rows(); @@ -1353,28 +1382,11 @@ class FileFormatScanNodeMixin : public FileFormatFixtureMixinV2, ASSERT_EQ(row_count, expected_rows()); } - void TestScanMissingFilterField() { - auto f32 = field("f32", float32()); - auto f64 = field("f64", float64()); - this->SetDatasetSchema({f32, f64}); - this->SetScanProjectionRefs({"f64"}); - this->SetScanFilter(equal(field_ref("f32"), literal(0))); - - auto reader = this->GetRandomData(dataset_schema_); - auto source = this->MakeBufferSource(reader.get()); - auto fragment = this->MakeFragment(*source); - - // At the moment, all formats support this. CSV & JSON simply ignore - // the filter field entirely. Parquet filters with statistics which doesn't require - // loading columns. - // - // However, it seems valid that a format would reject this case as well. Perhaps it - // is not worth testing. - ASSERT_OK_AND_ASSIGN(std::unique_ptr scanner, - this->Scan(fragment)); - } - - void TestScanProjectedNested(bool fine_grained_selection = false) { + // Given a nested column (e.g. a struct column) we should be able to ask for only parts + // of the nested structure. Some formats (columnar ones) support this in the reader. In + // other formats (e.g. JSON) we are forced to load the entire structure into memory and + // then discard the not-needed parts + void TestScanProjectedNested() { // "struct1": { // "f32", // "i32" @@ -1399,15 +1411,10 @@ class FileFormatScanNodeMixin : public FileFormatFixtureMixinV2, this->SetScanFilter(greater_equal(field_ref(FieldRef("struct2", "i64")), literal(0))); std::shared_ptr physical_schema; - if (fine_grained_selection) { - // Some formats, like Parquet, let you pluck only a part of a complex type - physical_schema = schema( - {field("struct1", struct_({f32})), field("struct2", struct_({i64, struct1}))}); - } else { - // Otherwise, the entire top-level field is returned - physical_schema = schema({struct1, struct2}); - } - std::shared_ptr projected_schema = schema({ + // Some formats, like Parquet, let you pluck only a part of a complex type + physical_schema = schema( + {field("struct1", struct_({f32})), field("struct2", struct_({i64, struct1}))}); + std::shared_ptr expected_schema = schema({ field(".struct1.f32", float32()), field(".struct2.struct1", struct1->type()), field(".struct2.struct1.f32", float32()), @@ -1424,128 +1431,11 @@ class FileFormatScanNodeMixin : public FileFormatFixtureMixinV2, for (auto maybe_batch : *scanner) { ASSERT_OK_AND_ASSIGN(auto batch, maybe_batch); row_count += batch->num_rows(); - AssertSchemaEqual(*batch->schema(), *projected_schema, + AssertSchemaEqual(*batch->schema(), *expected_schema, /*check_metadata=*/false); } ASSERT_EQ(row_count, expected_rows()); } - { - // File includes a duplicated name in struct2 - auto struct2_physical = field("struct2", struct_({f64, i64, struct1, i64})); - auto reader = - this->GetRandomData(schema({struct1, struct2_physical, f32, f64, i32, i64})); - auto source = this->MakeBufferSource(reader.get()); - auto fragment = this->MakeFragment(*source); - - ASSERT_OK_AND_ASSIGN(std::unique_ptr scanner, - this->Scan(fragment)); - EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, ::testing::HasSubstr("i64"), - scanner->Next().status()); - } - { - // File is missing a child in struct1 - auto struct1_physical = field("struct1", struct_({i32})); - auto reader = - this->GetRandomData(schema({struct1_physical, struct2, f32, f64, i32, i64})); - auto source = this->MakeBufferSource(reader.get()); - auto fragment = this->MakeFragment(*source); - - physical_schema = schema({physical_schema->field(1)}); - - int64_t row_count = 0; - ASSERT_OK_AND_ASSIGN(std::unique_ptr scanner, - this->Scan(fragment)); - for (auto maybe_batch : *scanner) { - ASSERT_OK_AND_ASSIGN(auto batch, maybe_batch); - row_count += batch->num_rows(); - ASSERT_THAT( - batch->schema()->fields(), - ::testing::UnorderedPointwise(PointeesEqual(), physical_schema->fields())) - << "EXPECTED:\n" - << physical_schema->ToString() << "\nACTUAL:\n" - << batch->schema()->ToString(); - } - ASSERT_EQ(row_count, expected_rows()); - } - } - - void TestScanProjectedMissingCols() { - auto f32 = field("f32", float32()); - auto f64 = field("f64", float64()); - auto i32 = field("i32", int32()); - auto i64 = field("i64", int64()); - this->SetDatasetSchema({f64, i64, f32, i32}); - this->SetScanProjectionRefs({"f64", "i32"}); - this->SetScanFilter(equal(field_ref("i32"), literal(0))); - - auto data_without_i32 = this->GetRandomData(schema({f64, i64, f32})); - auto data_without_f64 = this->GetRandomData(schema({i64, f32, i32})); - auto data_with_all = this->GetRandomData(schema({f64, i64, f32, i32})); - - auto readers = {data_with_all.get(), data_without_i32.get(), data_without_f64.get()}; - for (auto reader : readers) { - SCOPED_TRACE(reader->schema()->ToString()); - auto source = this->MakeBufferSource(reader); - auto fragment = this->MakeFragment(*source); - - // in the case where a file doesn't contain a referenced field, we materialize it - // as nulls - std::shared_ptr expected_schema = schema({f64, i32}); - - int64_t row_count = 0; - ASSERT_OK_AND_ASSIGN(std::unique_ptr scanner, - this->Scan(fragment)); - for (auto maybe_batch : *scanner) { - ASSERT_OK_AND_ASSIGN(auto batch, maybe_batch); - row_count += batch->num_rows(); - ASSERT_THAT( - batch->schema()->fields(), - ::testing::UnorderedPointwise(PointeesEqual(), expected_schema->fields())) - << "EXPECTED:\n" - << expected_schema->ToString() << "\nACTUAL:\n" - << batch->schema()->ToString(); - } - ASSERT_EQ(row_count, expected_rows()); - } - } - - void TestScanWithDuplicateColumn() { - // A duplicate column is ignored if not requested. - auto i32 = field("i32", int32()); - auto i64 = field("i64", int64()); - this->SetDatasetSchema({i32, i32, i64}); - this->SetScanProjectionRefs({"i64"}); - auto expected_schema = schema({i64}); - auto reader = this->GetRandomData(dataset_schema_); - auto source = this->MakeBufferSource(reader.get()); - auto fragment = this->MakeFragment(*source); - - int64_t row_count = 0; - - ASSERT_OK_AND_ASSIGN(std::unique_ptr scanner, - this->Scan(fragment)); - for (auto maybe_batch : *scanner) { - ASSERT_OK_AND_ASSIGN(auto batch, maybe_batch); - row_count += batch->num_rows(); - AssertSchemaEqual(*batch->schema(), *expected_schema, - /*check_metadata=*/false); - } - - ASSERT_EQ(row_count, expected_rows()); - - // Duplicate columns ok if column selection uses paths - row_count = 0; - expected_schema = schema({i32, i32}); - this->SetScanProjection({{0}, {1}}); - ASSERT_OK_AND_ASSIGN(scanner, this->Scan(fragment)); - for (auto maybe_batch : *scanner) { - ASSERT_OK_AND_ASSIGN(auto batch, maybe_batch); - row_count += batch->num_rows(); - AssertSchemaEqual(*batch->schema(), *expected_schema, - /*check_metadata=*/false); - } - - ASSERT_EQ(row_count, expected_rows()); } void TestScanWithPushdownNulls() { @@ -1572,6 +1462,25 @@ class FileFormatScanNodeMixin : public FileFormatFixtureMixinV2, ASSERT_EQ(row_count, 1); } + void TestInspect() { + auto f32 = field("f32", float32()); + auto f64 = field("f64", float64()); + auto i32 = field("i32", int32()); + auto i64 = field("i64", int64()); + std::shared_ptr test_schema = schema({f32, f64, i32, i64}); + + auto reader = this->GetRandomData(test_schema); + auto source = this->MakeBufferSource(reader.get()); + std::shared_ptr fragment = this->MakeFragment(*source); + + ASSERT_FINISHES_OK_AND_ASSIGN( + std::shared_ptr inspection, + fragment->InspectFragment(/*scan_options=*/{}, compute::threaded_exec_context(), + /*should_cache=false*/ false)); + + ASSERT_EQ(inspection->column_names, test_schema->field_names()); + } + protected: virtual const FragmentScanOptions* GetFormatOptions() = 0; diff --git a/cpp/src/arrow/testing/matchers.h b/cpp/src/arrow/testing/matchers.h index b4625b3922e86..7e02a1b5ebde3 100644 --- a/cpp/src/arrow/testing/matchers.h +++ b/cpp/src/arrow/testing/matchers.h @@ -264,7 +264,7 @@ class ErrorMatcher { } else if (status.ok()) { *listener << "whose non-error doesn't match"; } else { - *listener << "whose error doesn't match"; + *listener << "whose error (" << status.message() << ") doesn't match"; } testing::internal::PrintIfNotEmpty(value_listener.str(), listener->stream()); diff --git a/cpp/src/arrow/util/async_util.cc b/cpp/src/arrow/util/async_util.cc index 55627eb43bbcf..45c7723b7d70d 100644 --- a/cpp/src/arrow/util/async_util.cc +++ b/cpp/src/arrow/util/async_util.cc @@ -65,6 +65,9 @@ class ThrottleImpl : public ThrottledAsyncTaskScheduler::Throttle { void Pause() override { std::lock_guard lg(mutex_); + if (paused_) { + return; + } paused_ = true; if (!backoff_.is_valid()) { backoff_ = Future<>::Make(); @@ -73,6 +76,9 @@ class ThrottleImpl : public ThrottledAsyncTaskScheduler::Throttle { void Resume() override { std::unique_lock lk(mutex_); + if (!paused_) { + return; + } paused_ = false; // Might be a useless notification if our current cost is full // or no one is waiting but it should be ok. diff --git a/cpp/src/arrow/util/async_util.h b/cpp/src/arrow/util/async_util.h index 2668ae222609b..aa0be9978937b 100644 --- a/cpp/src/arrow/util/async_util.h +++ b/cpp/src/arrow/util/async_util.h @@ -131,16 +131,19 @@ class ARROW_EXPORT AsyncTaskScheduler { /// This visits the task serially without readahead. If readahead or parallelism /// is desired then it should be added in the generator itself. /// - /// The generator itself will be kept alive until all tasks have been completed. - /// However, if the scheduler is aborted, the generator will be destroyed as soon as the - /// next item would be requested. + /// The generator itself will be kept alive until all tasks have been completed and the + /// on_completion function is run. However, if the scheduler is aborted, the generator + /// will be destroyed as soon as the next item would be requested and the on_completion + /// function will not run. /// /// \param generator the generator to submit to the scheduler /// \param visitor a function which visits each generator future as it completes /// \param name a name which will be used for each submitted task + /// \param on_completion an optional function that will be called after all tasks template bool AddAsyncGenerator(std::function()> generator, - std::function visitor, std::string_view name); + std::function visitor, std::string_view name, + std::function on_completion = {}); template struct SimpleTask : public Task { @@ -270,12 +273,16 @@ class ARROW_EXPORT ThrottledAsyncTaskScheduler : public AsyncTaskScheduler { /// Pause the throttle /// /// Any tasks that have been submitted already will continue. However, no new tasks - /// will be run until the throttle is resumed. + /// will be run until the throttle is resumed.\ + /// + /// This method should be idempotent virtual void Pause() = 0; /// Resume the throttle /// /// Allows taks to be submitted again. If there is a max_concurrent_cost limit then /// it will still apply. + /// + /// This method should be idempotent virtual void Resume() = 0; /// Create a throttled view of a scheduler @@ -377,7 +384,8 @@ ARROW_EXPORT std::unique_ptr MakeThrottledAsyncTask template bool AsyncTaskScheduler::AddAsyncGenerator(std::function()> generator, std::function visitor, - std::string_view name) { + std::string_view name, + std::function on_completion) { struct State { State(std::function()> generator, std::function visitor, std::unique_ptr task_group, std::string_view name) @@ -412,8 +420,10 @@ bool AsyncTaskScheduler::AddAsyncGenerator(std::function()> generator, task_completion.MarkFinished(std::move(visit_st)); return; } - state_holder->task_group->AddTask( - std::make_unique(std::move(state_holder))); + State* state_view = state_holder.get(); + std::unique_ptr next_task = + std::make_unique(std::move(state_holder)); + state_view->task_group->AddTask(std::move(next_task)); task_completion.MarkFinished(); } std::unique_ptr state_holder; @@ -443,8 +453,12 @@ bool AsyncTaskScheduler::AddAsyncGenerator(std::function()> generator, std::unique_ptr state_holder; }; + std::function finish_cb = std::move(on_completion); + if (!finish_cb) { + finish_cb = [] { return Status::OK(); }; + } std::unique_ptr task_group = - AsyncTaskGroup::Make(this, [] { return Status::OK(); }); + AsyncTaskGroup::Make(this, std::move(finish_cb)); AsyncTaskGroup* task_group_view = task_group.get(); std::unique_ptr state_holder = std::make_unique( std::move(generator), std::move(visitor), std::move(task_group), name); diff --git a/cpp/src/parquet/arrow/reader.cc b/cpp/src/parquet/arrow/reader.cc index 855fb5a5a4882..aea05d92b3f20 100644 --- a/cpp/src/parquet/arrow/reader.cc +++ b/cpp/src/parquet/arrow/reader.cc @@ -16,10 +16,12 @@ // under the License. #include "parquet/arrow/reader.h" +#include #include #include #include +#include #include #include #include @@ -58,6 +60,7 @@ using arrow::Future; using arrow::Int32Array; using arrow::ListArray; using arrow::MemoryPool; +using arrow::RecordBatch; using arrow::RecordBatchReader; using arrow::ResizableBuffer; using arrow::Result; @@ -326,6 +329,23 @@ class FileReaderImpl : public FileReader { return ReadRowGroups(row_groups, Iota(reader_->metadata()->num_columns()), table); } + Result DoReadRowGroupsAsync( + const std::vector& row_groups, const std::vector& indices, + ::arrow::internal::Executor* cpu_executor, bool allow_sliced_batches); + + AsyncBatchGenerator ReadRowGroupsAsync(const std::vector& row_groups, + const std::vector& indices, + ::arrow::internal::Executor* cpu_executor, + bool allow_sliced_batches) override { + Result batch_gen = + DoReadRowGroupsAsync(row_groups, indices, cpu_executor, allow_sliced_batches); + if (batch_gen.ok()) { + return batch_gen.MoveValueUnsafe(); + } + return ::arrow::MakeFailingGenerator>( + batch_gen.status()); + } + Status ReadRowGroup(int row_group_index, const std::vector& column_indices, std::shared_ptr
* out) override { return ReadRowGroups({row_group_index}, column_indices, out); @@ -335,6 +355,28 @@ class FileReaderImpl : public FileReader { return ReadRowGroup(i, Iota(reader_->metadata()->num_columns()), table); } + AsyncBatchGenerator ReadRowGroupAsync(int row_group_index, + ::arrow::internal::Executor* cpu_executor, + bool allow_sliced_batches) override { + return ReadRowGroupsAsync({row_group_index}, Iota(reader_->metadata()->num_columns()), + cpu_executor, allow_sliced_batches); + } + + AsyncBatchGenerator ReadRowGroupAsync(int row_group_index, + const std::vector& column_indices, + ::arrow::internal::Executor* cpu_executor, + bool allow_sliced_batches) override { + return ReadRowGroupsAsync({row_group_index}, column_indices, cpu_executor, + allow_sliced_batches); + } + + AsyncBatchGenerator ReadRowGroupsAsync(const std::vector& row_groups, + ::arrow::internal::Executor* cpu_executor, + bool allow_sliced_batches) override { + return ReadRowGroupsAsync(row_groups, Iota(reader_->metadata()->num_columns()), + cpu_executor, allow_sliced_batches); + } + Status GetRecordBatchReader(const std::vector& row_group_indices, const std::vector& column_indices, std::unique_ptr* out) override; @@ -1243,6 +1285,124 @@ Status FileReaderImpl::ReadRowGroups(const std::vector& row_groups, return Status::OK(); } +struct AsyncBatchGeneratorState { + ::arrow::internal::Executor* io_executor; + ::arrow::internal::Executor* cpu_executor; + std::vector> column_readers; + std::queue> overflow; + std::shared_ptr<::arrow::Schema> schema; + int64_t batch_size; + int64_t rows_remaining; + bool use_threads; + bool allow_sliced_batches; +}; + +class AsyncBatchGeneratorImpl { + public: + explicit AsyncBatchGeneratorImpl(std::shared_ptr state) + : state_(std::move(state)) {} + Future> operator()() { + if (!state_->overflow.empty()) { + std::shared_ptr next = std::move(state_->overflow.front()); + state_->overflow.pop(); + return next; + } + + if (state_->rows_remaining == 0) { + // Exhausted + return Future>::MakeFinished( + ::arrow::IterationEnd>()); + } + + int64_t rows_in_batch = std::min(state_->rows_remaining, state_->batch_size); + state_->rows_remaining -= rows_in_batch; + + // We read the columns in parallel. Each reader returns a chunked array. This is + // probably because we might need to chunk a column if that column is too large. We + // do provide a batch size but perhaps that column has massive strings or something + // like that. + Future>> chunked_arrays_fut = + ::arrow::internal::OptionalParallelForAsync( + state_->use_threads, state_->column_readers, + [rows_in_batch](std::size_t, std::shared_ptr column_reader) + -> Result> { + std::shared_ptr chunked_array; + ARROW_RETURN_NOT_OK( + column_reader->NextBatch(rows_in_batch, &chunked_array)); + return chunked_array; + }); + + // Grab the first batch of data and return it. If there is more than one batch then + // throw the reamining batches into overflow and they will be fetched on the next call + return chunked_arrays_fut.Then( + [state = state_, + rows_in_batch](const std::vector>& chunks) + -> Result> { + std::shared_ptr
table = + Table::Make(state->schema, chunks, rows_in_batch); + ::arrow::TableBatchReader batch_reader(*table); + std::shared_ptr first; + while (true) { + ARROW_ASSIGN_OR_RAISE(std::shared_ptr next_batch, + batch_reader.Next()); + if (!next_batch) { + break; + } + if (first) { + if (!state->allow_sliced_batches) { + return Status::Invalid( + "The setting allow_sliced_batches is set to false and data was " + "encountered that was too large to fit in a single batch."); + } + state->overflow.push(std::move(next_batch)); + } else { + first = std::move(next_batch); + } + } + if (!first) { + // TODO(weston): Test this case + return Status::Invalid("Unexpected empty row group"); + } + return first; + }); + } + + private: + std::shared_ptr state_; +}; + +Result FileReaderImpl::DoReadRowGroupsAsync( + const std::vector& row_groups, const std::vector& column_indices, + ::arrow::internal::Executor* cpu_executor, bool allow_sliced_batches) { + RETURN_NOT_OK(BoundsCheck(row_groups, column_indices)); + + if (reader_properties_.pre_buffer()) { + BEGIN_PARQUET_CATCH_EXCEPTIONS + parquet_reader()->PreBuffer(row_groups, column_indices, + reader_properties_.io_context(), + reader_properties_.cache_options()); + END_PARQUET_CATCH_EXCEPTIONS + } + + auto generator_state = std::make_shared(); + generator_state->io_executor = reader_properties_.io_context().executor(); + generator_state->cpu_executor = cpu_executor; + generator_state->use_threads = reader_properties_.use_threads(); + generator_state->allow_sliced_batches = allow_sliced_batches; + RETURN_NOT_OK(GetFieldReaders(column_indices, row_groups, + &generator_state->column_readers, + &generator_state->schema)); + + generator_state->batch_size = properties().batch_size(); + generator_state->rows_remaining = 0; + for (int row_group : row_groups) { + generator_state->rows_remaining += + parquet_reader()->metadata()->RowGroup(row_group)->num_rows(); + } + + return AsyncBatchGeneratorImpl(std::move(generator_state)); +} + Future> FileReaderImpl::DecodeRowGroups( std::shared_ptr self, const std::vector& row_groups, const std::vector& column_indices, ::arrow::internal::Executor* cpu_executor) { diff --git a/cpp/src/parquet/arrow/reader.h b/cpp/src/parquet/arrow/reader.h index 2cbd36176f5e3..8442f9fb25a44 100644 --- a/cpp/src/parquet/arrow/reader.h +++ b/cpp/src/parquet/arrow/reader.h @@ -256,6 +256,71 @@ class PARQUET_EXPORT FileReader { virtual ::arrow::Status ReadRowGroups(const std::vector& row_groups, std::shared_ptr<::arrow::Table>* out) = 0; + using AsyncBatchGenerator = + std::function<::arrow::Future>()>; + + /// \brief Read a single row group from the file + /// + /// \see ReadRowGroupsAsync for operation details + /// + /// \param i the index of the row group to read + /// \param cpu_executor an executor to use to run CPU tasks + virtual AsyncBatchGenerator ReadRowGroupAsync(int i, + ::arrow::internal::Executor* cpu_executor, + bool allow_sliced_batches = false) = 0; + /// \brief Read some columns from a single row group from the file + /// + /// \see ReadRowGroupsAsync for operation details + /// \see ReadTable for details on how column indices are resolved + /// + /// \param i the index of the row group to read + /// \param column_indices leaf-indices of the columns to read + /// \param cpu_executor an executor to use to run CPU tasks + virtual AsyncBatchGenerator ReadRowGroupAsync(int i, + const std::vector& column_indices, + ::arrow::internal::Executor* cpu_executor, + bool allow_sliced_batches = false) = 0; + + /// \brief Read row groups from the file + /// + /// \see ReadRowGroupsAsync for operation details + /// + /// \param row_groups indices of the row groups to read + /// \param cpu_executor an executor to use to run CPU tasks + virtual AsyncBatchGenerator ReadRowGroupsAsync( + const std::vector& row_groups, ::arrow::internal::Executor* cpu_executor, + bool allow_sliced_batches = false) = 0; + + /// \brief Read some columns from the given rows groups from the file + /// + /// If pre-buffering is enabled then all of the data will be read using the pre-buffer + /// cache. See ParquetFileReader::PreBuffer for details on how this affects memory and + /// performance. + /// + /// This operation is not perfectly async. The read from disk will be done on an I/O + /// thread, which is correct. However, compression and column decoding is also done on + /// the I/O thread which may not be ideal. The stage after that (transferring the + /// decoded data into Arrow structures and fulfilling the future) should be done as a + /// new task on the cpu_executor. + /// + /// The returned generator will respect the batch size set in the reader properties. + /// Batches will not be larger than the given batch size. However, batches may be + /// smaller. This can happen, for example, when there is not enough data or when a + /// string column is too large to fit into a single batch. The parameter + /// `allow_sliced_batches` can be set to false to disallow this later case. This can be + /// useful when you need to know exactly how many batches you will get from a scan + /// before you start. + /// + /// The I/O executor is obtained from the I/O context in the reader properties. + /// + /// \param row_groups indices of the row groups to read + /// \param column_indices indices of the columns to read + /// \param cpu_executor an executor to use to run CPU tasks + /// \param allow_sliced_batches indicates whether or not we can slice large batches + virtual AsyncBatchGenerator ReadRowGroupsAsync( + const std::vector& row_groups, const std::vector& column_indices, + ::arrow::internal::Executor* cpu_executor, bool allow_sliced_batches = false) = 0; + /// \brief Scan file contents with one thread, return number of rows virtual ::arrow::Status ScanContents(std::vector columns, const int32_t column_batch_size, From 01776eca3fa952fa6c7b38eaa243339c1ccbdf64 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 30 May 2023 15:10:01 -0700 Subject: [PATCH 02/13] WIP --- cpp/src/arrow/dataset/scan_node.cc | 2 + cpp/src/arrow/dataset/scanner_test.cc | 128 ++++++++++++++++++++++---- 2 files changed, 110 insertions(+), 20 deletions(-) diff --git a/cpp/src/arrow/dataset/scan_node.cc b/cpp/src/arrow/dataset/scan_node.cc index dd16a7787d480..5b9c6313cd3bf 100644 --- a/cpp/src/arrow/dataset/scan_node.cc +++ b/cpp/src/arrow/dataset/scan_node.cc @@ -167,6 +167,8 @@ class ScanTaskStagingArea { ScanTaskEntry scan_task; scan_task.scan_task_index = i; scan_task.num_batches = this->fragment_scanner->NumBatchesInScanTask(i); + std::cout << " Fragment " << fragment_index << " detected " + << scan_task.num_batches << " batches in scan task " << i << std::endl; scan_task.launched = false; scan_tasks.push_back(scan_task); } diff --git a/cpp/src/arrow/dataset/scanner_test.cc b/cpp/src/arrow/dataset/scanner_test.cc index 4a1fa01fbfb9a..85e7e00b7ce33 100644 --- a/cpp/src/arrow/dataset/scanner_test.cc +++ b/cpp/src/arrow/dataset/scanner_test.cc @@ -162,13 +162,24 @@ TEST(BasicEvolution, ReorderedColumns) { } struct MockScanTask { - explicit MockScanTask(std::shared_ptr batch) : batch(std::move(batch)) {} + void FinishNext() { Finish(current_index++); } - void Finish() { batch_future.MarkFinished(batch); } + void Finish(std::size_t batch_idx) { + if (batch_idx >= batches.size()) { + FAIL() << "MockScanTask::FinishNext called " << batches.size() + << " times but a batch at index " << batch_idx << " was asked for"; + } + batch_futures[batch_idx].MarkFinished(batches[batch_idx]); + } + + void AddBatch(std::shared_ptr batch) { + batches.push_back(std::move(batch)); + batch_futures.push_back(Future>::Make()); + } - std::shared_ptr batch; - Future> batch_future = - Future>::Make(); + std::vector> batches; + std::vector>> batch_futures; + std::size_t current_index = 0; }; // Wraps access to std::default_random_engine to ensure only one thread @@ -213,30 +224,50 @@ struct MockFragmentScanner : public FragmentScanner { } // ### Unit Test API ### - void DeliverBatches(bool slow, const std::vector& to_deliver) { - for (MockScanTask task : to_deliver) { + struct ItemToDeliver { + std::size_t scan_task_number; + std::size_t batch_number; + }; + + void DeliverBatches(bool slow, const std::vector& to_deliver) { + for (const auto& task : to_deliver) { if (slow) { - std::ignore = SleepABitAsync().Then( - [task]() mutable { task.batch_future.MarkFinished(task.batch); }); + std::ignore = SleepABitAsync().Then([this, task]() mutable { + scan_tasks_[task.scan_task_number].Finish(task.batch_number); + }); } else { - task.batch_future.MarkFinished(task.batch); + scan_tasks_[task.scan_task_number].Finish(task.batch_number); } } } - void DeliverBatchesInOrder(bool slow) { DeliverBatches(slow, scan_tasks_); } + std::vector AllBatchesInOrder() { + std::vector items; + ItemToDeliver current; + for (std::size_t task_idx = 0; task_idx < scan_tasks_.size(); task_idx++) { + current.scan_task_number = task_idx; + for (std::size_t batch_idx = 0; batch_idx < scan_tasks_[task_idx].batches.size(); + batch_idx++) { + current.batch_number = batch_idx; + items.push_back(current); + } + } + return items; + } + + void DeliverBatchesInOrder(bool slow) { DeliverBatches(slow, AllBatchesInOrder()); } void DeliverBatchesRandomly(bool slow, ConcurrentGen* gen) { - std::vector shuffled_tasks(scan_tasks_); + std::vector shuffled_tasks = AllBatchesInOrder(); gen->With([&](std::default_random_engine* gen_instance) { std::shuffle(shuffled_tasks.begin(), shuffled_tasks.end(), *gen_instance); }); - DeliverBatches(slow, shuffled_tasks); + DeliverBatches(slow, std::move(shuffled_tasks)); } - bool HasStarted(int batch_number) { return has_started_[batch_number]; } - bool HasDelivered(int batch_number) { - return scan_tasks_[batch_number].batch_future.is_finished(); + bool HasStartedScanTask(int task_number) { return has_started_[task_number]; } + bool HasDeliveredBatch(int task_number, int batch_number) { + return scan_tasks_[task_number].batch_futures[batch_number].is_finished(); } int num_batches() { return static_cast(scan_tasks_.size()); } @@ -414,11 +445,22 @@ struct MockDatasetBuilder { active_fragment = fragments[fragments.size() - 1]->fragment_scanner_.get(); } - void AddBatch(std::shared_ptr batch) { - active_fragment->scan_tasks_.emplace_back(std::move(batch)); + void AddScanTask() { + if (active_fragment == nullptr) { + FAIL() << "you must call AddFragment before calling AddScanTask"; + } + active_scan_task_index = active_fragment->scan_tasks_.size(); + active_fragment->scan_tasks_.emplace_back(); active_fragment->has_started_.push_back(false); } + void AddBatch(std::shared_ptr batch) { + if (active_scan_task_index < 0) { + FAIL() << "you must call AddScanTask before calling AddBatch"; + } + active_fragment->scan_tasks_[active_scan_task_index].AddBatch(std::move(batch)); + } + std::unique_ptr Finish() { return std::make_unique(std::move(dataset_schema), std::move(fragments)); } @@ -426,6 +468,7 @@ struct MockDatasetBuilder { std::shared_ptr dataset_schema; std::vector> fragments; MockFragmentScanner* active_fragment = nullptr; + std::size_t active_scan_task_index = -1; }; template MakeTestBatch(int idx) { return RecordBatch::Make(ScannerTestSchema(), kRowsPerTestBatch, std::move(arrays)); } -std::unique_ptr MakeTestDataset(int num_fragments, int batches_per_fragment, +std::unique_ptr MakeTestDataset(int num_fragments, + int scan_tasks_per_fragment, + int batches_per_scan_task, bool empty = false) { std::shared_ptr test_schema = ScannerTestSchema(); MockDatasetBuilder dataset_builder(test_schema); @@ -513,7 +558,7 @@ std::unique_ptr MakeTestDataset(int num_fragments, int batches_per_ dataset_builder.AddFragment( test_schema, std::make_unique(test_schema->field_names()), Fragment::kNoPartitionInformation); - for (int j = 0; j < batches_per_fragment; j++) { + for (int j = 0; j < scan_tasks_per_fragment; j++) { if (empty) { dataset_builder.AddBatch( RecordBatch::Make(schema({}), kRowsPerTestBatch, ArrayVector{})); @@ -727,6 +772,49 @@ TEST(TestNewScanner, OutOfOrderFragmentCompletion) { ASSERT_EQ(3, batches.size()); } +TEST(TestNewScanner, OutOfOrderScanTaskCompletion) { + constexpr int kNumFragments = 1; + constexpr int kNumScanTasks = 4; + constexpr int kNumBatchesPerScanTask = 1; + + internal::Initialize(); + std::shared_ptr test_dataset = + MakeTestDataset(kNumFragments, kNumBatchesPerFragment); + + ScanV2Options options(test_dataset); + options.columns = ScanV2Options::AllColumns(*test_dataset->schema()); + + // Begin scan + acero::Declaration scan_decl = acero::Declaration("scan2", std::move(options)); + Future batches_fut = + acero::DeclarationToBatchesAsync(std::move(scan_decl)); + + // Start scanning on all fragments + for (int i = 0; i < kNumFragments; i++) { + test_dataset->fragments_[i]->FinishInspection(); + test_dataset->fragments_[i]->FinishScanBegin(); + } + + // Let scan tasks get started + SleepABit(); + // A fragment in the middle finishes + test_dataset->fragments_[1]->fragment_scanner_->scan_tasks_[0].Finish(); + + SleepABit(); + + // A fragment at the end finishes + test_dataset->fragments_[2]->fragment_scanner_->scan_tasks_[0].Finish(); + + SleepABit(); + + // Now the first fragment finishes + test_dataset->fragments_[0]->fragment_scanner_->scan_tasks_[0].Finish(); + + // The scan should finish cleanly + ASSERT_FINISHES_OK_AND_ASSIGN(RecordBatchVector batches, batches_fut); + ASSERT_EQ(3, batches.size()); +} + TEST(TestNewScanner, NestedRead) { // This tests the case where the file format does not support // handling nested reads (e.g. JSON) and so the scanner must From dfcc90762656251876c79752ba9f473308cbee56 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Wed, 31 May 2023 09:13:47 -0700 Subject: [PATCH 03/13] WIP --- cpp/src/arrow/dataset/file_parquet.cc | 46 ++++- cpp/src/arrow/dataset/scan_node.cc | 42 +---- cpp/src/arrow/dataset/scanner_test.cc | 233 ++++++++++++++++---------- cpp/src/arrow/type.cc | 22 +++ cpp/src/arrow/type.h | 4 + 5 files changed, 217 insertions(+), 130 deletions(-) diff --git a/cpp/src/arrow/dataset/file_parquet.cc b/cpp/src/arrow/dataset/file_parquet.cc index fec2bf1e7bb38..0622001115a2a 100644 --- a/cpp/src/arrow/dataset/file_parquet.cc +++ b/cpp/src/arrow/dataset/file_parquet.cc @@ -476,7 +476,48 @@ class ParquetFragmentScanner : public FragmentScanner { return Status::OK(); } - Future<> Initialize(std::shared_ptr file) { + // We need to convert from a list of paths ([2, 1, 0], [0, 3]) to a list of leaf indices + // + // First we sort the paths in DFS order(well, get the sorted path indices) so the above + // example would change to something like [[0, 3], [2, 1, 0]] + // + // Then we walk through the schema in DFS fashion, counting leaf nodes, until we + // encounter the next item in the sorted list + // + // 0 - Not leaf + // 0 0 - leaf_counter=0 + // 0 1 - leaf_counter=1 + // 0 2 - Not leaf + // 0 2 0 - leaf_counter=2 + // 0 2 1 - leaf_counter=3 + // 0 3 - leaf_counter=4* + // + // Here we mark down 4 for the [0, 3] element. Keep in mind that the eventual return + // value will have the 4 in position 1 of the returned list (to maintain order) + // + // If we encounter a non-leaf node then we need to load all leaves under that node + void CalculateDesiredColumns(const FragmentScanRequest& request) { + std::vector sorted_path_indices( + request.fragment_selection->columns().size()); + std::iota(sorted_path_indices.begin(), sorted_path_indices.end(), 0); + std::sort(sorted_path_indices.begin(), sorted_path_indices.end(), + [&](const auto& lhs, const auto& rhs) { + return request.fragment_selection->columns()[lhs].path < + request.fragment_selection->columns()[rhs].path; + }); + for (const auto& column : request.fragment_selection->columns()) { + sorted_paths.push_back(column.path); + } + std::vector desired_columns_.reserve( + request.fragment_selection->columns().size()); + + for (const auto& column : request.fragment_selection->columns()) { + desired_columns_.push_back(PathToLeafIndex(column.path)); + } + } + + Future<> Initialize(std::shared_ptr file, + const FragmentScanRequest& request) { parquet::ReaderProperties properties = MakeParquetReaderProperties(); // TODO(ARROW-12259): workaround since we have Future<(move-only type)> auto reader_fut = parquet::ParquetFileReader::OpenAsync( @@ -515,7 +556,7 @@ class ParquetFragmentScanner : public FragmentScanner { std::make_shared(std::move(path), inspection.file_metadata, scan_options, format_reader_options, exec_context); - return parquet_fragment_scanner->Initialize(std::move(file)) + return parquet_fragment_scanner->Initialize(std::move(file), request) .Then([fragment_scanner = std::static_pointer_cast( parquet_fragment_scanner)]() { return fragment_scanner; }); } @@ -529,6 +570,7 @@ class ParquetFragmentScanner : public FragmentScanner { compute::ExecContext* exec_context_; // These are set during Initialize + std::vector desired_columns_; std::unique_ptr file_reader_; int32_t batch_size_; }; diff --git a/cpp/src/arrow/dataset/scan_node.cc b/cpp/src/arrow/dataset/scan_node.cc index 5b9c6313cd3bf..eaaed5e4a21cc 100644 --- a/cpp/src/arrow/dataset/scan_node.cc +++ b/cpp/src/arrow/dataset/scan_node.cc @@ -17,7 +17,6 @@ #include #include -#include #include #include #include @@ -108,7 +107,6 @@ class ScanTaskStagingArea { void InsertInspectedFragment(std::shared_ptr fragment_scanner, int fragment_index, std::unique_ptr task_launcher) { - std::cout << " Inserting fragment into staging area" << std::endl; auto fragment_entry = std::make_unique( std::move(fragment_scanner), fragment_index, std::move(task_launcher)); @@ -146,7 +144,6 @@ class ScanTaskStagingArea { std::lock_guard lg(mutex_); total_num_fragments_ = num_fragments; if (num_fragments_processed_ == total_num_fragments_) { - std::cout << " Ending on finish" << std::endl; completion_cb_(); } } @@ -167,8 +164,6 @@ class ScanTaskStagingArea { ScanTaskEntry scan_task; scan_task.scan_task_index = i; scan_task.num_batches = this->fragment_scanner->NumBatchesInScanTask(i); - std::cout << " Fragment " << fragment_index << " detected " - << scan_task.num_batches << " batches in scan task " << i << std::endl; scan_task.launched = false; scan_tasks.push_back(scan_task); } @@ -198,13 +193,7 @@ class ScanTaskStagingArea { } void TryAndLaunchTasksUnlocked() { - if (!root_) { - std::cout << " Can't launch tasks because there are none" << std::endl; - return; - } - if (num_scan_tasks_running_ >= run_limit_) { - std::cout << " Can't even start trying to run tasks because we hit the limit" - << std::endl; + if (!root_ || num_scan_tasks_running_ >= run_limit_) { return; } FragmentEntry* itr = root_.get(); @@ -212,13 +201,9 @@ class ScanTaskStagingArea { while (itr != nullptr) { for (auto& scan_task : itr->scan_tasks) { if (!scan_task.launched) { - std::cout << " Launching a scan task frag=" << itr->fragment_index - << std::endl; scan_task.launched = true; LaunchScanTaskUnlocked(itr, scan_task.scan_task_index, num_preceding_batches); if (num_scan_tasks_running_ >= run_limit_) { - std::cout << " Can't launch more tasks because we hit the limit" - << std::endl; // We've launched as many as we can return; } @@ -226,8 +211,6 @@ class ScanTaskStagingArea { if (scan_task.num_batches >= 0) { num_preceding_batches += scan_task.num_batches; } else { - std::cout << " Can't launch task because waiting for troublemaker" - << std::endl; // A scan task is running that doesn't know how many batches it has. We can't // proceed return; @@ -242,19 +225,17 @@ class ScanTaskStagingArea { std::lock_guard lg(mutex_); batches_completed_ += num_batches; num_scan_tasks_running_--; - std::cout << " MarkScanTaskFinished(batches_completed_=" << batches_completed_ << ")" - << std::endl; - const auto& itr = fragment_entry->scan_tasks.cbegin(); + auto itr = fragment_entry->scan_tasks.cbegin(); std::size_t old_size = fragment_entry->scan_tasks.size(); while (itr != fragment_entry->scan_tasks.cend()) { if (itr->scan_task_index == scan_task_number) { fragment_entry->scan_tasks.erase(itr); break; } + itr++; } DCHECK_LT(fragment_entry->scan_tasks.size(), old_size); if (fragment_entry->scan_tasks.empty()) { - std::cout << "Fragment complete: " << fragment_entry->fragment_index << std::endl; FragmentEntry* prev = fragment_entry->prev; if (prev == nullptr) { // The current root has finished @@ -279,7 +260,6 @@ class ScanTaskStagingArea { } num_fragments_processed_++; if (num_fragments_processed_ == total_num_fragments_) { - std::cout << " Completion via fragment end" << std::endl; completion_cb_(); return; } @@ -476,20 +456,16 @@ class ScanNode : public acero::ExecNode, public acero::TracedNode { ->async_scheduler() ->AddAsyncGenerator>( std::move(batch_gen), - [this, scan_task_number, batch_counter = batch_count.get()]( - const std::shared_ptr& batch) { - std::cout << " ScanTask(" << scan_task_number << "," << (*batch_counter) - << ")" << std::endl; + [this, batch_counter = + batch_count.get()](const std::shared_ptr& batch) { (*batch_counter)++; return HandleBatch(batch); }, "ScanNode::ScanBatch::Next", - [this, scan_task_number, task_complete_cb = std::move(task_complete_cb), + [this, task_complete_cb = std::move(task_complete_cb), batch_count = std::move(batch_count)]() { node_->plan_->query_context()->ScheduleTask( - [scan_task_number, task_complete_cb, batch_count] { - std::cout << " Scan task complete(" << scan_task_number << ")" - << std::endl; + [task_complete_cb, batch_count] { task_complete_cb(*batch_count); return Status::OK(); }, @@ -553,7 +529,6 @@ class ScanNode : public acero::ExecNode, public acero::TracedNode { } Result> operator()() override { - std::cout << "START: Inspecting fragment: " << fragment_->ToString() << std::endl; return fragment_ ->InspectFragment(node_->options_.format_options, node_->plan_->query_context()->exec_context(), @@ -629,7 +604,6 @@ class ScanNode : public acero::ExecNode, public acero::TracedNode { CreateTaskLauncher(std::move(fragment_evolution), extracted.remaining_columns, filter_minus_part, std::move(extracted.known_values))); - std::cout << " BeginScan: " << fragment_->ToString() << std::endl; return fragment_ ->BeginScan(task_launcher->scan_request(), *inspected_fragment, node_->plan_->query_context()->exec_context()) @@ -674,7 +648,6 @@ class ScanNode : public acero::ExecNode, public acero::TracedNode { ->AddAsyncGenerator>( std::move(frag_gen), [this](const std::shared_ptr& fragment) { - std::cout << " Listed fragment: " << fragment->ToString() << std::endl; inspection_throttle_->AddTask(std::make_unique( this, fragment, fragment_index_++)); return Status::OK(); @@ -702,7 +675,6 @@ class ScanNode : public acero::ExecNode, public acero::TracedNode { std::move(completion)); plan_->query_context()->async_scheduler()->AddSimpleTask( [this] { - std::cout << "START: ListFragments::GetFragments" << std::endl; return GetFragments(options_.dataset.get(), options_.filter) .Then([this](const AsyncGenerator>& frag_gen) { InspectFragments(frag_gen); diff --git a/cpp/src/arrow/dataset/scanner_test.cc b/cpp/src/arrow/dataset/scanner_test.cc index 85e7e00b7ce33..0e33eb718a375 100644 --- a/cpp/src/arrow/dataset/scanner_test.cc +++ b/cpp/src/arrow/dataset/scanner_test.cc @@ -205,22 +205,22 @@ struct MockFragmentScanner : public FragmentScanner { AsyncGenerator> RunScanTask( int scan_task_number) override { auto batch_number = std::make_shared(0); - return [this, batch_number] { - if (*batch_number == static_cast(scan_tasks_.size())) { + has_started_[scan_task_number] = true; + return [this, batch_number, scan_task_number] { + auto& scan_task = scan_tasks_[scan_task_number]; + if (*batch_number == static_cast(scan_task.batches.size())) { return AsyncGeneratorEnd>(); } - has_started_[*batch_number] = true; - Future> batch = - scan_tasks_[*batch_number].batch_future; + Future> batch = scan_task.batch_futures[*batch_number]; (*batch_number)++; return batch; }; } - int NumScanTasks() override { return 1; } + int NumScanTasks() override { return static_cast(scan_tasks_.size()); } int NumBatchesInScanTask(int task_number) override { - return static_cast(scan_tasks_.size()); + return static_cast(scan_tasks_[task_number].batches.size()); } // ### Unit Test API ### @@ -349,11 +349,11 @@ struct MockFragment : public Fragment { bool has_inspected() { return has_inspected_; } bool has_started() { return has_started_; } - bool HasBatchStarted(int batch_index) { - return fragment_scanner_->HasStarted(batch_index); + bool HasStartedScanTask(int scan_task_number) { + return fragment_scanner_->HasStartedScanTask(scan_task_number); } - bool HasBatchDelivered(int batch_index) { - return fragment_scanner_->HasDelivered(batch_index); + bool HasDeliveredBatch(int scan_task_number, int batch_index) { + return fragment_scanner_->HasDeliveredBatch(scan_task_number, batch_index); } std::shared_ptr fragment_scanner_; @@ -420,8 +420,8 @@ struct MockDataset : public FragmentDataset { bool HasStartedFragment(int fragment_index) { return fragments_[fragment_index]->has_started(); } - bool HasStartedBatch(int fragment_index, int batch_index) { - return fragments_[fragment_index]->HasBatchStarted(batch_index); + bool HasStartedScanTask(int fragment_index, int scan_task_number) { + return fragments_[fragment_index]->HasStartedScanTask(scan_task_number); } bool has_started_ = false; @@ -449,7 +449,7 @@ struct MockDatasetBuilder { if (active_fragment == nullptr) { FAIL() << "you must call AddFragment before calling AddScanTask"; } - active_scan_task_index = active_fragment->scan_tasks_.size(); + active_scan_task_index = static_cast(active_fragment->scan_tasks_.size()); active_fragment->scan_tasks_.emplace_back(); active_fragment->has_started_.push_back(false); } @@ -468,7 +468,7 @@ struct MockDatasetBuilder { std::shared_ptr dataset_schema; std::vector> fragments; MockFragmentScanner* active_fragment = nullptr; - std::size_t active_scan_task_index = -1; + int active_scan_task_index = -1; }; template ArrayFromRange(int start, int end, bool add_nulls) { struct ScannerTestParams { bool slow; int num_fragments; + int num_scan_tasks; int num_batches; std::string ToString() const { std::stringstream ss; - ss << (slow ? "slow" : "fast") << num_fragments << "f" << num_batches << "b"; + ss << (slow ? "slow" : "fast") << num_fragments << "f" << num_scan_tasks << "st" + << num_batches << "b"; return ss.str(); } @@ -507,8 +509,10 @@ struct ScannerTestParams { static std::vector Values() { std::vector values; for (bool slow : {false, true}) { - values.push_back({slow, 1, 128}); - values.push_back({slow, 16, 128}); + values.push_back({slow, 1, 1, 128}); + values.push_back({slow, 16, 1, 128}); + values.push_back({slow, 1, 4, 32}); + values.push_back({slow, 16, 4, 32}); } return values; } @@ -516,7 +520,7 @@ struct ScannerTestParams { std::ostream& operator<<(std::ostream& out, const ScannerTestParams& params) { out << (params.slow ? "slow-" : "fast-") << params.num_fragments << "f-" - << params.num_batches << "b"; + << params.num_scan_tasks << "st-" << params.num_batches << "b"; return out; } @@ -559,11 +563,16 @@ std::unique_ptr MakeTestDataset(int num_fragments, test_schema, std::make_unique(test_schema->field_names()), Fragment::kNoPartitionInformation); for (int j = 0; j < scan_tasks_per_fragment; j++) { - if (empty) { - dataset_builder.AddBatch( - RecordBatch::Make(schema({}), kRowsPerTestBatch, ArrayVector{})); - } else { - dataset_builder.AddBatch(MakeTestBatch(i * batches_per_fragment + j)); + dataset_builder.AddScanTask(); + for (int k = 0; k < batches_per_scan_task; k++) { + if (empty) { + dataset_builder.AddBatch( + RecordBatch::Make(schema({}), kRowsPerTestBatch, ArrayVector{})); + } else { + dataset_builder.AddBatch( + MakeTestBatch(i * scan_tasks_per_fragment * batches_per_scan_task + + j * batches_per_scan_task + k)); + } } } } @@ -577,8 +586,13 @@ class TestScannerBase : public ::testing::TestWithParam { std::shared_ptr MakeExpectedBatch() { RecordBatchVector batches; for (int frag_idx = 0; frag_idx < GetParam().num_fragments; frag_idx++) { - for (int batch_idx = 0; batch_idx < GetParam().num_batches; batch_idx++) { - batches.push_back(MakeTestBatch(batch_idx + (frag_idx * GetParam().num_batches))); + for (int scan_task_idx = 0; scan_task_idx < GetParam().num_scan_tasks; + scan_task_idx++) { + for (int batch_idx = 0; batch_idx < GetParam().num_batches; batch_idx++) { + batches.push_back(MakeTestBatch( + batch_idx + (scan_task_idx * GetParam().num_batches) + + (frag_idx * GetParam().num_batches * GetParam().num_scan_tasks))); + } } } EXPECT_OK_AND_ASSIGN(std::shared_ptr
table, @@ -623,8 +637,8 @@ class TestScannerBase : public ::testing::TestWithParam { } void CheckScanner(bool ordered) { - std::shared_ptr mock_dataset = - MakeTestDataset(GetParam().num_fragments, GetParam().num_batches); + std::shared_ptr mock_dataset = MakeTestDataset( + GetParam().num_fragments, GetParam().num_scan_tasks, GetParam().num_batches); acero::Declaration scan_decl = MakeScanNode(mock_dataset); RecordBatchVector scanned_batches = RunNode(scan_decl, ordered, mock_dataset.get()); CheckScannedBatches(std::move(scanned_batches)); @@ -643,7 +657,7 @@ INSTANTIATE_TEST_SUITE_P(BasicNewScannerTests, TestScannerBase, }); void CheckScannerBackpressure(std::shared_ptr dataset, ScanV2Options options, - int maxConcurrentScanTasks, + int max_concurrent_scan_tasks, ::arrow::internal::ThreadPool* thread_pool) { // These are hard-coded and not configurable constexpr int kMaxInspectionsReadahead = 4; @@ -667,49 +681,71 @@ void CheckScannerBackpressure(std::shared_ptr dataset, ScanV2Option BusyWait(10, [&] { return get_num_inspected() >= static_cast(kMaxInspectionsReadahead); }); - std::cout << "### Wait done. Ensuring inspections readahead limit reached ###" - << std::endl; SleepABit(); ASSERT_EQ(get_num_inspected(), static_cast(kMaxInspectionsReadahead)); - std::cout << "### Assert passed. Finishing inspections and starting scan ###" - << std::endl; - int total_batches = 0; + int total_scan_tasks = 0; for (const auto& frag : dataset->fragments_) { - total_batches += frag->fragment_scanner_->num_batches(); + total_scan_tasks += frag->fragment_scanner_->scan_tasks_.size(); frag->FinishInspection(); frag->FinishScanBegin(); } - int batches_scanned = 0; - while (batches_scanned < total_batches) { - MockScanTask* next_task_to_deliver = nullptr; + // Batches haven't been delivered yet so scan tasks can't complete so we should only + // see max_concurrent_scan_tasks scan tasks running + thread_pool->WaitForIdle(); + + int num_scan_tasks_started = 0; + for (const auto& frag : dataset->fragments_) { + for (const auto& started : frag->fragment_scanner_->has_started_) { + if (started) { + num_scan_tasks_started++; + } + } + } + ASSERT_EQ(max_concurrent_scan_tasks, num_scan_tasks_started); + + // Deliver each scan task one at a time, ensuring that we never run more than + // max_concurrent_scan_tasks + int fragment_index = 0; + int scan_task_index = 0; + int scan_tasks_run = 0; + while (scan_tasks_run < total_scan_tasks) { thread_pool->WaitForIdle(); - int batches_started = 0; + int scan_tasks_started = 0; for (const auto& frag : dataset->fragments_) { - for (int i = 0; i < frag->fragment_scanner_->num_batches(); i++) { - if (frag->HasBatchStarted(i)) { - batches_started++; - if (next_task_to_deliver == nullptr && !frag->HasBatchDelivered(i)) { - next_task_to_deliver = &frag->fragment_scanner_->scan_tasks_[i]; - } + for (int i = 0; i < static_cast(frag->fragment_scanner_->has_started_.size()); + i++) { + if (frag->HasStartedScanTask(i)) { + scan_tasks_started++; } } } - ASSERT_LE(batches_started - batches_scanned, maxConcurrentScanTasks) + ASSERT_LE(scan_tasks_started - scan_tasks_run, max_concurrent_scan_tasks) << " too many scan tasks were allowed to run"; - ASSERT_NE(next_task_to_deliver, nullptr); - next_task_to_deliver->batch_future.MarkFinished(next_task_to_deliver->batch); - batches_scanned++; + MockScanTask& task_to_deliver = dataset->fragments_[fragment_index] + ->fragment_scanner_->scan_tasks_[scan_task_index]; + for (std::size_t batch_idx = 0; batch_idx < task_to_deliver.batches.size(); + batch_idx++) { + task_to_deliver.Finish(batch_idx); + } + scan_tasks_run++; + scan_task_index++; + if (scan_task_index == + dataset->fragments_[fragment_index]->fragment_scanner_->NumScanTasks()) { + scan_task_index = 0; + fragment_index++; + } } } TEST(TestNewScanner, Backpressure) { - constexpr int kNumFragments = 20; - constexpr int kNumBatchesPerFragment = 4; + constexpr int kNumFragments = 4; + constexpr int kNumScanTasksPerFragment = 4; + constexpr int kNumBatchesPerScanTask = 4; internal::Initialize(); std::shared_ptr test_dataset = - MakeTestDataset(kNumFragments, kNumBatchesPerFragment); + MakeTestDataset(kNumFragments, kNumScanTasksPerFragment, kNumBatchesPerScanTask); ScanV2Options options(test_dataset); @@ -721,7 +757,8 @@ TEST(TestNewScanner, Backpressure) { ::arrow::internal::GetCpuThreadPool()); // Some readahead - test_dataset = MakeTestDataset(kNumFragments, kNumBatchesPerFragment); + test_dataset = + MakeTestDataset(kNumFragments, kNumScanTasksPerFragment, kNumBatchesPerScanTask); options = ScanV2Options(test_dataset); options.columns = ScanV2Options::AllColumns(*test_dataset->schema()); options.scan_task_readahead = 4; @@ -732,11 +769,12 @@ TEST(TestNewScanner, Backpressure) { // Test a fragment completing when a previous fragment is still running TEST(TestNewScanner, OutOfOrderFragmentCompletion) { constexpr int kNumFragments = 3; + constexpr int kNumScanTasksPerFragment = 1; constexpr int kNumBatchesPerFragment = 1; internal::Initialize(); std::shared_ptr test_dataset = - MakeTestDataset(kNumFragments, kNumBatchesPerFragment); + MakeTestDataset(kNumFragments, kNumScanTasksPerFragment, kNumBatchesPerFragment); ScanV2Options options(test_dataset); options.columns = ScanV2Options::AllColumns(*test_dataset->schema()); @@ -755,65 +793,65 @@ TEST(TestNewScanner, OutOfOrderFragmentCompletion) { // Let scan tasks get started SleepABit(); // A fragment in the middle finishes - test_dataset->fragments_[1]->fragment_scanner_->scan_tasks_[0].Finish(); + test_dataset->fragments_[1]->fragment_scanner_->scan_tasks_[0].Finish(0); SleepABit(); // A fragment at the end finishes - test_dataset->fragments_[2]->fragment_scanner_->scan_tasks_[0].Finish(); + test_dataset->fragments_[2]->fragment_scanner_->scan_tasks_[0].Finish(0); SleepABit(); // Now the first fragment finishes - test_dataset->fragments_[0]->fragment_scanner_->scan_tasks_[0].Finish(); + test_dataset->fragments_[0]->fragment_scanner_->scan_tasks_[0].Finish(0); // The scan should finish cleanly ASSERT_FINISHES_OK_AND_ASSIGN(RecordBatchVector batches, batches_fut); ASSERT_EQ(3, batches.size()); } -TEST(TestNewScanner, OutOfOrderScanTaskCompletion) { - constexpr int kNumFragments = 1; - constexpr int kNumScanTasks = 4; - constexpr int kNumBatchesPerScanTask = 1; +// TEST(TestNewScanner, OutOfOrderScanTaskCompletion) { +// constexpr int kNumFragments = 1; +// constexpr int kNumScanTasksPerFragment = 4; +// constexpr int kNumBatchesPerScanTask = 1; - internal::Initialize(); - std::shared_ptr test_dataset = - MakeTestDataset(kNumFragments, kNumBatchesPerFragment); +// internal::Initialize(); +// std::shared_ptr test_dataset = +// MakeTestDataset(kNumFragments, kNumScanTasksPerFragment, kNumBatchesPerScanTask); - ScanV2Options options(test_dataset); - options.columns = ScanV2Options::AllColumns(*test_dataset->schema()); +// ScanV2Options options(test_dataset); +// options.columns = ScanV2Options::AllColumns(*test_dataset->schema()); - // Begin scan - acero::Declaration scan_decl = acero::Declaration("scan2", std::move(options)); - Future batches_fut = - acero::DeclarationToBatchesAsync(std::move(scan_decl)); +// // Begin scan +// acero::Declaration scan_decl = acero::Declaration("scan2", std::move(options)); +// Future batches_fut = +// acero::DeclarationToBatchesAsync(std::move(scan_decl)); - // Start scanning on all fragments - for (int i = 0; i < kNumFragments; i++) { - test_dataset->fragments_[i]->FinishInspection(); - test_dataset->fragments_[i]->FinishScanBegin(); - } +// // Start scanning on all fragments +// for (int i = 0; i < kNumFragments; i++) { +// test_dataset->fragments_[i]->FinishInspection(); +// test_dataset->fragments_[i]->FinishScanBegin(); +// } - // Let scan tasks get started - SleepABit(); - // A fragment in the middle finishes - test_dataset->fragments_[1]->fragment_scanner_->scan_tasks_[0].Finish(); +// // Let scan tasks get started +// SleepABit(); +// // A fragment in the middle finishes +// test_dataset->fragments_[1]->fragment_scanner_->scan_tasks_[0].Finish(0); - SleepABit(); +// SleepABit(); - // A fragment at the end finishes - test_dataset->fragments_[2]->fragment_scanner_->scan_tasks_[0].Finish(); +// // A fragment at the end finishes +// test_dataset->fragments_[2]->fragment_scanner_->scan_tasks_[0].Finish(0); - SleepABit(); +// SleepABit(); - // Now the first fragment finishes - test_dataset->fragments_[0]->fragment_scanner_->scan_tasks_[0].Finish(); +// // Now the first fragment finishes +// test_dataset->fragments_[0]->fragment_scanner_->scan_tasks_[0].Finish(0); - // The scan should finish cleanly - ASSERT_FINISHES_OK_AND_ASSIGN(RecordBatchVector batches, batches_fut); - ASSERT_EQ(3, batches.size()); -} +// // The scan should finish cleanly +// ASSERT_FINISHES_OK_AND_ASSIGN(RecordBatchVector batches, batches_fut); +// ASSERT_EQ(3, batches.size()); +// } TEST(TestNewScanner, NestedRead) { // This tests the case where the file format does not support @@ -823,6 +861,7 @@ TEST(TestNewScanner, NestedRead) { std::shared_ptr test_schema = ScannerTestSchema(); MockDatasetBuilder builder(test_schema); builder.AddFragment(test_schema); + builder.AddScanTask(); std::shared_ptr batch = MakeTestBatch(0); ASSERT_OK_AND_ASSIGN(std::shared_ptr nested_col, FieldPath({2, 0}).Get(*batch)); std::shared_ptr one_column = RecordBatch::Make( @@ -853,11 +892,13 @@ std::shared_ptr MakePartitionSkipDataset() { MockDatasetBuilder builder(test_schema); builder.AddFragment(test_schema, /*inspection=*/nullptr, equal(field_ref({1}), literal(100))); + builder.AddScanTask(); std::shared_ptr batch = MakeTestBatch(0); EXPECT_OK_AND_ASSIGN(batch, batch->RemoveColumn(1)); builder.AddBatch(std::move(batch)); builder.AddFragment(test_schema, /*inspection=*/nullptr, equal(field_ref({1}), literal(50))); + builder.AddScanTask(); batch = MakeTestBatch(1); EXPECT_OK_AND_ASSIGN(batch, batch->RemoveColumn(1)); builder.AddBatch(std::move(batch)); @@ -873,6 +914,7 @@ std::shared_ptr MakeInvalidPartitionSkipDataset() { MockDatasetBuilder builder(test_schema); builder.AddFragment(test_schema, /*inspection=*/nullptr, equal(field_ref({1}), literal(100))); + builder.AddScanTask(); std::shared_ptr batch = MakeTestBatch(0); EXPECT_OK_AND_ASSIGN(batch, batch->RemoveColumn(1)); builder.AddBatch(std::move(batch)); @@ -951,6 +993,7 @@ TEST(TestNewScanner, EmptyFragment) { std::shared_ptr test_schema = ScannerTestSchema(); MockDatasetBuilder builder(test_schema); builder.AddFragment(test_schema); + builder.AddScanTask(); std::shared_ptr test_dataset = builder.Finish(); test_dataset->DeliverBatchesInOrder(false); @@ -966,6 +1009,7 @@ TEST(TestNewScanner, EmptyBatch) { std::shared_ptr test_schema = ScannerTestSchema(); MockDatasetBuilder builder(test_schema); builder.AddFragment(test_schema); + builder.AddScanTask(); ASSERT_OK_AND_ASSIGN(std::shared_ptr empty_batch, RecordBatch::MakeEmpty(test_schema)); builder.AddBatch(std::move(empty_batch)); @@ -981,16 +1025,18 @@ TEST(TestNewScanner, EmptyBatch) { TEST(TestNewScanner, NoColumns) { constexpr int kNumFragments = 4; - constexpr int kNumBatchesPerFragment = 4; + constexpr int kNumScanTasksPerFragment = 4; + constexpr int kNumBatchesPerScanTask = 4; internal::Initialize(); - std::shared_ptr test_dataset = - MakeTestDataset(kNumFragments, kNumBatchesPerFragment, /*empty=*/true); + std::shared_ptr test_dataset = MakeTestDataset( + kNumFragments, kNumScanTasksPerFragment, kNumBatchesPerScanTask, /*empty=*/true); test_dataset->DeliverBatchesInOrder(false); ScanV2Options options(test_dataset); ASSERT_OK_AND_ASSIGN(acero::BatchesWithCommonSchema batches_and_schema, acero::DeclarationToExecBatches({"scan2", options})); - ASSERT_EQ(16, batches_and_schema.batches.size()); + ASSERT_EQ(kNumFragments * kNumScanTasksPerFragment * kNumBatchesPerScanTask, + batches_and_schema.batches.size()); for (const auto& batch : batches_and_schema.batches) { ASSERT_EQ(0, batch.values.size()); ASSERT_EQ(kRowsPerTestBatch, batch.length); @@ -1005,6 +1051,7 @@ TEST(TestNewScanner, MissingColumn) { ASSERT_OK_AND_ASSIGN(std::shared_ptr missing_schema, test_schema->RemoveField(2)); builder.AddFragment(missing_schema); + builder.AddScanTask(); std::shared_ptr batch = MakeTestBatch(0); // Remove column 2 because we are pretending it doesn't exist // in the fragment diff --git a/cpp/src/arrow/type.cc b/cpp/src/arrow/type.cc index 68dc2aabe96ad..859f7ebd742fd 100644 --- a/cpp/src/arrow/type.cc +++ b/cpp/src/arrow/type.cc @@ -1053,6 +1053,28 @@ size_t FieldPath::hash() const { return internal::ComputeStringHash<0>(indices().data(), indices().size() * sizeof(int)); } +bool FieldPath::operator<(const FieldPath& other) const { + std::size_t idx; + for (idx = 0; idx < indices().size(); idx++) { + if (idx >= other.indices_.size()) { + // *this is something like [2, 5, 0] and other is something like [2, 5] + return false; + } + if (indices_[idx] < other.indices_[idx]) { + return true; + } + if (indices_[idx] > other.indices_[idx]) { + return false; + } + } + if (idx < other.indices_.size()) { + // *this is something like [2, 5] and other is something like [2, 5, 0] + return true; + } + // Equality + return false; +} + std::string FieldPath::ToString() const { if (this->indices().empty()) { return "FieldPath(empty)"; diff --git a/cpp/src/arrow/type.h b/cpp/src/arrow/type.h index ccd1ddccf54ce..7b6399e734e94 100644 --- a/cpp/src/arrow/type.h +++ b/cpp/src/arrow/type.h @@ -1673,6 +1673,10 @@ class ARROW_EXPORT FieldPath { bool empty() const { return indices_.empty(); } bool operator==(const FieldPath& other) const { return indices() == other.indices(); } bool operator!=(const FieldPath& other) const { return indices() != other.indices(); } + bool operator<(const FieldPath& other) const; + bool operator>(const FieldPath& other) { return other < *this; } + bool operator<=(const FieldPath& other) { return !(*this > other); } + bool operator>=(const FieldPath& other) { return !(*this < other); } const std::vector& indices() const { return indices_; } int operator[](size_t i) const { return indices_[i]; } From 83ab0275401cca971c2278fc5e6ccc06360dfa83 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Fri, 2 Jun 2023 06:14:16 -0700 Subject: [PATCH 04/13] WIP --- cpp/src/arrow/dataset/file_parquet.cc | 42 ++------------------------- cpp/src/arrow/util/async_util.h | 2 +- 2 files changed, 3 insertions(+), 41 deletions(-) diff --git a/cpp/src/arrow/dataset/file_parquet.cc b/cpp/src/arrow/dataset/file_parquet.cc index 0622001115a2a..78c419161e26a 100644 --- a/cpp/src/arrow/dataset/file_parquet.cc +++ b/cpp/src/arrow/dataset/file_parquet.cc @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -51,6 +52,7 @@ #include "parquet/file_reader.h" #include "parquet/properties.h" #include "parquet/statistics.h" +#include "schema.h" namespace arrow { @@ -476,46 +478,6 @@ class ParquetFragmentScanner : public FragmentScanner { return Status::OK(); } - // We need to convert from a list of paths ([2, 1, 0], [0, 3]) to a list of leaf indices - // - // First we sort the paths in DFS order(well, get the sorted path indices) so the above - // example would change to something like [[0, 3], [2, 1, 0]] - // - // Then we walk through the schema in DFS fashion, counting leaf nodes, until we - // encounter the next item in the sorted list - // - // 0 - Not leaf - // 0 0 - leaf_counter=0 - // 0 1 - leaf_counter=1 - // 0 2 - Not leaf - // 0 2 0 - leaf_counter=2 - // 0 2 1 - leaf_counter=3 - // 0 3 - leaf_counter=4* - // - // Here we mark down 4 for the [0, 3] element. Keep in mind that the eventual return - // value will have the 4 in position 1 of the returned list (to maintain order) - // - // If we encounter a non-leaf node then we need to load all leaves under that node - void CalculateDesiredColumns(const FragmentScanRequest& request) { - std::vector sorted_path_indices( - request.fragment_selection->columns().size()); - std::iota(sorted_path_indices.begin(), sorted_path_indices.end(), 0); - std::sort(sorted_path_indices.begin(), sorted_path_indices.end(), - [&](const auto& lhs, const auto& rhs) { - return request.fragment_selection->columns()[lhs].path < - request.fragment_selection->columns()[rhs].path; - }); - for (const auto& column : request.fragment_selection->columns()) { - sorted_paths.push_back(column.path); - } - std::vector desired_columns_.reserve( - request.fragment_selection->columns().size()); - - for (const auto& column : request.fragment_selection->columns()) { - desired_columns_.push_back(PathToLeafIndex(column.path)); - } - } - Future<> Initialize(std::shared_ptr file, const FragmentScanRequest& request) { parquet::ReaderProperties properties = MakeParquetReaderProperties(); diff --git a/cpp/src/arrow/util/async_util.h b/cpp/src/arrow/util/async_util.h index aa0be9978937b..09a483899e3aa 100644 --- a/cpp/src/arrow/util/async_util.h +++ b/cpp/src/arrow/util/async_util.h @@ -273,7 +273,7 @@ class ARROW_EXPORT ThrottledAsyncTaskScheduler : public AsyncTaskScheduler { /// Pause the throttle /// /// Any tasks that have been submitted already will continue. However, no new tasks - /// will be run until the throttle is resumed.\ + /// will be run until the throttle is resumed. /// /// This method should be idempotent virtual void Pause() = 0; From 85d72a21125a170be4df6c029260c2459c87d3e6 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Fri, 2 Jun 2023 07:06:03 -0700 Subject: [PATCH 05/13] WIP --- cpp/src/arrow/acero/sink_node_test.cc | 77 +++++++++++++++++++++++++++ cpp/src/arrow/dataset/file_parquet.h | 2 +- cpp/src/arrow/dataset/scan_node.cc | 6 +-- 3 files changed, 81 insertions(+), 4 deletions(-) create mode 100644 cpp/src/arrow/acero/sink_node_test.cc diff --git a/cpp/src/arrow/acero/sink_node_test.cc b/cpp/src/arrow/acero/sink_node_test.cc new file mode 100644 index 0000000000000..53fe2f0ea94b9 --- /dev/null +++ b/cpp/src/arrow/acero/sink_node_test.cc @@ -0,0 +1,77 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include +#include + +#include +#include + +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/dataset/file_base.h" +#include "arrow/dataset/file_parquet.h" +#include "arrow/dataset/partition.h" +#include "arrow/filesystem/mockfs.h" +#include "arrow/testing/generator.h" +#include "arrow/testing/gtest_util.h" +#include "arrow/testing/matchers.h" + +#include "arrow/table.h" +#include "arrow/util/key_value_metadata.h" + +namespace arrow { + +namespace acero { + +TEST(SinkNode, CustomFieldMetadata) { + // Create an input table with a nullable and a non-nullable type + ExecBatch batch = gen::Gen({gen::Step()})->FailOnError()->ExecBatch(/*num_rows=*/1); + std::shared_ptr test_schema = + schema({field("nullable_i32", uint32(), /*nullable=*/true, + key_value_metadata({{"foo", "bar"}})), + field("non_nullable_i32", uint32(), /*nullable=*/false)}); + std::shared_ptr record_batch = + RecordBatch::Make(test_schema, /*num_rows=*/1, + {batch.values[0].make_array(), batch.values[0].make_array()}); + ASSERT_OK_AND_ASSIGN(std::shared_ptr
table, + Table::FromRecordBatches({std::move(record_batch)})); + + ASSERT_TRUE(table->field(0)->nullable()); + ASSERT_EQ(1, table->field(0)->metadata()->keys().size()); + ASSERT_FALSE(table->field(1)->nullable()); + ASSERT_EQ(0, table->field(1)->metadata()->keys().size()); + + Declaration plan = Declaration::Sequence( + {{"table_source", TableSourceNodeOptions(std::move(table))}, + {"project", ProjectNodeOptions({compute::field_ref(0), compute::field_ref(1)})}}); + + ASSERT_OK_AND_ASSIGN(std::shared_ptr
out_table, DeclarationToTable(plan)); + + ASSERT_TRUE(table->field(0)->nullable()); + ASSERT_EQ(1, table->field(0)->metadata()->keys().size()); + ASSERT_FALSE(table->field(1)->nullable()); + ASSERT_EQ(0, table->field(1)->metadata()->keys().size()); + + ASSERT_OK_AND_ASSIGN(BatchesWithCommonSchema batches_and_schema, + DeclarationToExecBatches(plan)); + ASSERT_TRUE(batches_and_schema.schema->field(0)->nullable()); + ASSERT_FALSE(batches_and_schema.schema->field(1)->nullable()); +} + +} // namespace acero +} // namespace arrow diff --git a/cpp/src/arrow/dataset/file_parquet.h b/cpp/src/arrow/dataset/file_parquet.h index 93dc6df458476..08ad24c287cb5 100644 --- a/cpp/src/arrow/dataset/file_parquet.h +++ b/cpp/src/arrow/dataset/file_parquet.h @@ -247,7 +247,7 @@ class ARROW_DS_EXPORT ParquetFragmentScanOptions : public FragmentScanOptions { /// /// kCustom - Pre-buffering, stream buffering, and batch size settings will be /// configured according to reader_properties and arrow_reader_properties. - ParquetScanStrategy scan_strategy = ParquetScanStrategy::kMaxSpeed; + ParquetScanStrategy scan_strategy = ParquetScanStrategy::kLeastMemory; /// Reader properties. /// diff --git a/cpp/src/arrow/dataset/scan_node.cc b/cpp/src/arrow/dataset/scan_node.cc index eaaed5e4a21cc..ec65cb28a3924 100644 --- a/cpp/src/arrow/dataset/scan_node.cc +++ b/cpp/src/arrow/dataset/scan_node.cc @@ -452,13 +452,13 @@ class ScanNode : public acero::ExecNode, public acero::TracedNode { AsyncGenerator> batch_gen = scanner->RunScanTask(scan_task_number); auto batch_count = std::make_shared(0); + int* batch_count_view = batch_count.get(); node_->plan_->query_context() ->async_scheduler() ->AddAsyncGenerator>( std::move(batch_gen), - [this, batch_counter = - batch_count.get()](const std::shared_ptr& batch) { - (*batch_counter)++; + [this, batch_count_view](const std::shared_ptr& batch) { + (*batch_count_view)++; return HandleBatch(batch); }, "ScanNode::ScanBatch::Next", From e687c70e1a544d3e3c3eae98ca66209e8e85e5cb Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Fri, 14 Jul 2023 17:26:10 -0700 Subject: [PATCH 06/13] Reverting non-essential commit --- cpp/src/arrow/testing/matchers.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/testing/matchers.h b/cpp/src/arrow/testing/matchers.h index 7e02a1b5ebde3..b4625b3922e86 100644 --- a/cpp/src/arrow/testing/matchers.h +++ b/cpp/src/arrow/testing/matchers.h @@ -264,7 +264,7 @@ class ErrorMatcher { } else if (status.ok()) { *listener << "whose non-error doesn't match"; } else { - *listener << "whose error (" << status.message() << ") doesn't match"; + *listener << "whose error doesn't match"; } testing::internal::PrintIfNotEmpty(value_listener.str(), listener->stream()); From 80786f7209cb79390c70fca8b407c3cc31617612 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Mon, 17 Jul 2023 18:00:55 -0700 Subject: [PATCH 07/13] Added support for column projection --- cpp/src/arrow/dataset/file_parquet.cc | 197 +++++++++++++++++++-- cpp/src/arrow/dataset/file_parquet_test.cc | 5 +- cpp/src/arrow/dataset/test_util_internal.h | 105 +++++------ 3 files changed, 241 insertions(+), 66 deletions(-) diff --git a/cpp/src/arrow/dataset/file_parquet.cc b/cpp/src/arrow/dataset/file_parquet.cc index 78c419161e26a..cf5ce3d5c010d 100644 --- a/cpp/src/arrow/dataset/file_parquet.cc +++ b/cpp/src/arrow/dataset/file_parquet.cc @@ -17,6 +17,7 @@ #include "arrow/dataset/file_parquet.h" +#include #include #include #include @@ -37,6 +38,7 @@ #include "arrow/io/interfaces.h" #include "arrow/io/type_fwd.h" #include "arrow/table.h" +#include "arrow/util/async_generator.h" #include "arrow/util/async_generator_fwd.h" #include "arrow/util/bit_util.h" #include "arrow/util/checked_cast.h" @@ -68,19 +70,20 @@ using parquet::arrow::StatisticsAsScalars; namespace { -std::vector ColumnNamesFromMetadata( - const parquet::FileMetaData& file_metadata) { - std::vector names(file_metadata.num_columns()); - for (int i = 0; i < file_metadata.num_columns(); i++) { - names[i] = file_metadata.schema()->Column(i)->name(); +Result> ColumnNamesFromManifest(const SchemaManifest& manifest) { + std::vector names; + names.reserve(manifest.schema_fields.size()); + for (const auto& schema_field : manifest.schema_fields) { + names.push_back(schema_field.field->name()); } return names; } class ParquetInspectedFragment : public InspectedFragment { public: - explicit ParquetInspectedFragment(std::shared_ptr file_metadata) - : InspectedFragment(ColumnNamesFromMetadata(*file_metadata)), + explicit ParquetInspectedFragment(std::shared_ptr file_metadata, + std::vector column_names) + : InspectedFragment(std::move(column_names)), file_metadata(std::move(file_metadata)) {} std::shared_ptr file_metadata; @@ -352,8 +355,25 @@ class ParquetFragmentScanner : public FragmentScanner { exec_context_(exec_context) {} AsyncGenerator> RunScanTask(int task_number) override { - // TODO(weston) limit columns to read - return file_reader_->ReadRowGroupAsync(task_number, exec_context_->executor()); + AsyncGenerator> row_group_batches = + file_reader_->ReadRowGroupAsync(task_number, desired_columns_, + exec_context_->executor()); + return MakeMappedGenerator( + row_group_batches, + [this](const std::shared_ptr& batch) { return Reshape(batch); }); + } + + Result> Reshape( + const std::shared_ptr& batch) { + std::vector> arrays; + std::vector> fields; + for (const auto& path : selection_paths_) { + ARROW_ASSIGN_OR_RAISE(std::shared_ptr path_arr, path.Get(*batch)); + fields.push_back(field("", path_arr->type())); + arrays.push_back(std::move(path_arr)); + } + return RecordBatch::Make(schema(std::move(fields)), batch->num_rows(), + std::move(arrays)); } int NumScanTasks() override { return file_metadata_->num_row_groups(); } @@ -478,6 +498,145 @@ class ParquetFragmentScanner : public FragmentScanner { return Status::OK(); } + Result ResolvePath(const FieldPath& path, + const SchemaManifest& manifest) { + const SchemaField* itr; + DCHECK_LT(path[0], static_cast(manifest.schema_fields.size())); + itr = &manifest.schema_fields[path[0]]; + for (std::size_t i = 1; i < path.indices().size(); i++) { + // This should be guaranteed by evolution but maybe we should be more flexible + // to account for bugs in the evolution strategy? + DCHECK(!itr->is_leaf()); + DCHECK_LT(path[i], static_cast(itr->children.size())); + itr = &itr->children[path[i]]; + } + return itr; + } + + void AddColumnIndices(const SchemaField& field, std::vector* indices) { + if (field.is_leaf()) { + indices->push_back(field.column_index); + } + for (const auto& child : field.children) { + AddColumnIndices(child, indices); + } + } + + using SchemaFieldToPathMap = std::unordered_multimap; + // This struct is a recursive helper. See the comment on CalculateProjection for + // details. In here we walk through the file schema and calculate the expected output + // schema for a given set of column indices + struct PartialSchemaResolver { + bool Resolve(const std::vector& column_indices, + SchemaFieldToPathMap* node_to_paths, const SchemaField& field) { + bool field_is_included = false; + if (field.is_leaf()) { + if (field.column_index == next_included_index) { + field_is_included = true; + next_included_index_index++; + if (next_included_index_index < column_indices.size()) { + next_included_index = column_indices[next_included_index_index]; + } else { + next_included_index = -1; + } + } + } + int num_included_children = 0; + for (const auto& child : field.children) { + current_path.push_back(num_included_children); + if (Resolve(column_indices, node_to_paths, child)) { + num_included_children++; + field_is_included = true; + } + current_path.pop_back(); + } + auto found_paths = node_to_paths->equal_range(&field); + for (auto& item = found_paths.first; item != found_paths.second; item++) { + *item->second = FieldPath(current_path); + } + return field_is_included; + } + + Status Resolve(const std::vector& column_indices, + SchemaFieldToPathMap* node_to_paths, const SchemaManifest& manifest) { + next_included_index_index = 0; + next_included_index = column_indices[next_included_index_index]; + int num_included_fields = 0; + for (const auto& schema_field : manifest.schema_fields) { + current_path.push_back(num_included_fields); + const SchemaField* field = &schema_field; + if (Resolve(column_indices, node_to_paths, *field)) { + num_included_fields++; + } + current_path.pop_back(); + } + return Status::OK(); + } + + std::size_t next_included_index_index = 0; + int next_included_index = -1; + std::vector current_path; + }; + + // We are given `selection`, which is a vector of paths into the file schema. + // + // The parquet reader needs a vector of leaf indices. For non-nested fields this + // is easy. For nested fields it is a bit more complex. Consider the schema: + // + // A B C + // | + // E - - D + // | + // F - - G + // + // A and C are top-level flat fields. B and D are nested structs. The leaf indices + // are A(0), E(1), F(2), G(3), and C(4). + // + // If the user asks for field F we must include 2 in the indices we send to the file + // reader. A more challenging task is that we must also figure out the path to F in + // the batches the file reader will be returning. + // + // For example, if F is the only selection then {2} will be the only index we send to + // the file reader and returned batches will have a schema that includes B, D, and F. + // The path to F in those batches will be 0,0,0. If the user asks for A, E, and F then + // the returned schema will include A, B, E, D, and F. The path to F in those batches + // will be 1,1,0. + // + // If the user asks for a non-leaf field then we must make sure to add all leaf indices + // under that field. In the above example, if the user asks for D then we must add both + // F(2) and G(3) to the column indices we send to the parquet reader. + Status CalculateProjection(const parquet::FileMetaData& metadata, + const FragmentSelection& selection, + const SchemaManifest& manifest) { + if (selection.columns().empty()) { + return Status::OK(); + } + selection_paths_.resize(selection.columns().size()); + // A map from each node to the paths we need to fill in for that node + // (these will be items in selection_paths_) + std::unordered_multimap node_to_paths; + for (std::size_t column_idx = 0; column_idx < selection.columns().size(); + column_idx++) { + const FragmentSelectionColumn& selected_column = selection.columns()[column_idx]; + ARROW_ASSIGN_OR_RAISE(const SchemaField* selected_field, + ResolvePath(selected_column.path, manifest)); + AddColumnIndices(*selected_field, &desired_columns_); + node_to_paths.insert({selected_field, &selection_paths_[column_idx]}); + } + + // Sort the leaf indices and remove duplicates + std::sort(desired_columns_.begin(), desired_columns_.end()); + desired_columns_.erase(std::unique(desired_columns_.begin(), desired_columns_.end()), + desired_columns_.end()); + + // Now we calculate the expected output schema and use that to calculate the expected + // output paths. A node is included in this schema if there is at least one leaf + // under that node. + return PartialSchemaResolver().Resolve(desired_columns_, &node_to_paths, manifest); + // As we return we can expect that both desired_columns_ and selection_paths_ + // are properly initialized + } + Future<> Initialize(std::shared_ptr file, const FragmentScanRequest& request) { parquet::ReaderProperties properties = MakeParquetReaderProperties(); @@ -485,8 +644,8 @@ class ParquetFragmentScanner : public FragmentScanner { auto reader_fut = parquet::ParquetFileReader::OpenAsync( std::move(file), std::move(properties), file_metadata_); return reader_fut.Then( - [this, reader_fut]( - const std::unique_ptr&) mutable -> Status { + [this, reader_fut, + request](const std::unique_ptr&) mutable -> Status { ARROW_ASSIGN_OR_RAISE(std::unique_ptr reader, reader_fut.MoveResult()); std::shared_ptr metadata = reader->metadata(); @@ -500,6 +659,8 @@ class ParquetFragmentScanner : public FragmentScanner { RETURN_NOT_OK(parquet::arrow::FileReader::Make( exec_context_->memory_pool(), std::move(reader), std::move(arrow_properties), &file_reader_)); + RETURN_NOT_OK(CalculateProjection(*file_metadata_, *request.fragment_selection, + file_reader_->manifest())); return Status::OK(); }, [this](const Status& status) -> Status { @@ -533,6 +694,7 @@ class ParquetFragmentScanner : public FragmentScanner { // These are set during Initialize std::vector desired_columns_; + std::vector selection_paths_; std::unique_ptr file_reader_; int32_t batch_size_; }; @@ -648,10 +810,17 @@ Future> ParquetFileFormat::InspectFragment( ARROW_ASSIGN_OR_RAISE(auto input, source.Open()); auto reader_fut = parquet::ParquetFileReader::OpenAsync(std::move(input), std::move(properties)); - return reader_fut.Then([](const std::unique_ptr& reader) - -> std::shared_ptr { + return reader_fut.Then([this](const std::unique_ptr& reader) + -> Result> { std::shared_ptr metadata = reader->metadata(); - return std::make_shared(std::move(metadata)); + parquet::ArrowReaderProperties arrow_reader_props = + MakeArrowReaderProperties(reader_options, *metadata); + ARROW_ASSIGN_OR_RAISE(std::shared_ptr manifest, + GetSchemaManifest(*metadata, arrow_reader_props)); + ARROW_ASSIGN_OR_RAISE(std::vector column_names, + ColumnNamesFromManifest(*manifest)); + return std::make_shared(std::move(metadata), + std::move(column_names)); }); } diff --git a/cpp/src/arrow/dataset/file_parquet_test.cc b/cpp/src/arrow/dataset/file_parquet_test.cc index 01ca696a431a0..7c8e3f5094c9e 100644 --- a/cpp/src/arrow/dataset/file_parquet_test.cc +++ b/cpp/src/arrow/dataset/file_parquet_test.cc @@ -743,7 +743,10 @@ class TestParquetFileFormatScanNode TEST_P(TestParquetFileFormatScanNode, Inspect) { TestInspect(); } TEST_P(TestParquetFileFormatScanNode, Scan) { TestScan(); } -TEST_P(TestParquetFileFormatScanNode, ScanSomeColumn) { TestScanSomeColumns(); } +TEST_P(TestParquetFileFormatScanNode, ScanSomeColumns) { TestScanSomeColumns(); } +TEST_P(TestParquetFileFormatScanNode, ScanSomeNestedColumns) { + TestScanSomeNestedColumns(); +} TEST_P(TestParquetFileFormatScanNode, ScanWithInvalidOptions) { TestInvalidFormatScanOptions(); } diff --git a/cpp/src/arrow/dataset/test_util_internal.h b/cpp/src/arrow/dataset/test_util_internal.h index cb9e14ec97d2d..71bfc7ca6e030 100644 --- a/cpp/src/arrow/dataset/test_util_internal.h +++ b/cpp/src/arrow/dataset/test_util_internal.h @@ -1301,6 +1301,18 @@ class FileFormatScanNodeMixin : public FileFormatFixtureMixinV2, return reader; } + Result> ScanToTable(std::shared_ptr fragment, + bool add_filter_fields = true) { + ARROW_ASSIGN_OR_RAISE(std::unique_ptr scanner, + this->Scan(fragment, /*add_filter_fields=*/false)); + std::vector> batches; + for (auto maybe_batch : *scanner) { + ARROW_ASSIGN_OR_RAISE(auto batch, maybe_batch); + batches.push_back(batch); + } + return Table::FromRecordBatches(batches); + } + // Return a batch iterator which scans the fragment through the scanner. // // For interface compatibility with `FileFormatScanMixin` @@ -1345,7 +1357,8 @@ class FileFormatScanNodeMixin : public FileFormatFixtureMixinV2, // We create test data with 4 columns // We ask for only one of them (f64) // We set a row filter that relies on a different column (i32) - // This row filter matches all rows + // This row filter matches all rows (note, this row filter is just + // noise. Mainly making sure we aren't loading "materialized refs") // We expect to get back all rows and only the one column we asked for void TestScanSomeColumns() { auto f32 = field("f32", float32()); @@ -1382,60 +1395,50 @@ class FileFormatScanNodeMixin : public FileFormatFixtureMixinV2, ASSERT_EQ(row_count, expected_rows()); } - // Given a nested column (e.g. a struct column) we should be able to ask for only parts - // of the nested structure. Some formats (columnar ones) support this in the reader. In - // other formats (e.g. JSON) we are forced to load the entire structure into memory and - // then discard the not-needed parts - void TestScanProjectedNested() { - // "struct1": { - // "f32", - // "i32" - // } - // "struct2": { - // "f64", - // "i64", - // "struct1": { - // "f32", - // "i32" - // } - // } - auto f32 = field("f32", float32()); - auto f64 = field("f64", float64()); - auto i32 = field("i32", int32()); - auto i64 = field("i64", int64()); - auto struct1 = field("struct1", struct_({f32, i32})); - auto struct2 = field("struct2", struct_({f64, i64, struct1})); - this->SetDatasetSchema({struct1, struct2, f32, f64, i32, i64}); - this->SetScanProjectionRefs( - {".struct1.f32", ".struct2.struct1", ".struct2.struct1.f32"}); - this->SetScanFilter(greater_equal(field_ref(FieldRef("struct2", "i64")), literal(0))); - - std::shared_ptr physical_schema; - // Some formats, like Parquet, let you pluck only a part of a complex type - physical_schema = schema( - {field("struct1", struct_({f32})), field("struct2", struct_({i64, struct1}))}); - std::shared_ptr expected_schema = schema({ - field(".struct1.f32", float32()), - field(".struct2.struct1", struct1->type()), - field(".struct2.struct1.f32", float32()), - }); - - { + // For file formats that support nested columns we need to test various + // scanning scenarios where only part of a nested column is asked for. + // + // The schema is: + // + // A B C + // | + // E - - D + // | + // F - - G + void TestScanSomeNestedColumns() { + auto a = field("A", float32()); + auto c = field("C", float32()); + auto e = field("E", float32()); + auto f = field("F", float32()); + auto g = field("G", float32()); + auto d = field("D", struct_({f, g})); + auto b = field("B", struct_({e, d})); + this->SetDatasetSchema({a, b, c}); + + // Only F + this->SetScanProjection({{1, 1, 0}}); + auto expected_schema = schema({f}); + + auto CheckProjection = [&](std::vector projection, + std::shared_ptr expected_schema) { + this->SetScanProjection(projection); auto reader = this->GetRandomData(dataset_schema_); auto source = this->MakeBufferSource(reader.get()); auto fragment = this->MakeFragment(*source); + ASSERT_OK_AND_ASSIGN(std::shared_ptr
table, ScanToTable(fragment)); + AssertSchemaEqual(*table->schema(), *expected_schema, /*check_metadata=*/false); + }; - int64_t row_count = 0; - ASSERT_OK_AND_ASSIGN(std::unique_ptr scanner, - this->Scan(fragment)); - for (auto maybe_batch : *scanner) { - ASSERT_OK_AND_ASSIGN(auto batch, maybe_batch); - row_count += batch->num_rows(); - AssertSchemaEqual(*batch->schema(), *expected_schema, - /*check_metadata=*/false); - } - ASSERT_EQ(row_count, expected_rows()); - } + // Only F + CheckProjection({{1, 1, 0}}, schema({f})); + // A and F + CheckProjection({{0}, {1, 1, 0}}, schema({a, f})); + // Different order + CheckProjection({{1, 1, 0}, {0}}, schema({f, a})); + // Duplicates + CheckProjection({{0}, {1, 1, 0}, {0}}, schema({a, f, a})); + // // A and F and E + CheckProjection({{0}, {1, 1, 0}, {1, 0}}, schema({a, f, e})); } void TestScanWithPushdownNulls() { From 87f7b4e88c4bbca7440e0ecfbec071b8165c4077 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 18 Jul 2023 10:53:08 -0700 Subject: [PATCH 08/13] Added row group filtering --- cpp/src/arrow/dataset/dataset.cc | 4 +- cpp/src/arrow/dataset/dataset.h | 4 +- cpp/src/arrow/dataset/file_base.cc | 12 +- cpp/src/arrow/dataset/file_base.h | 8 +- cpp/src/arrow/dataset/file_csv.cc | 12 +- cpp/src/arrow/dataset/file_csv.h | 6 +- cpp/src/arrow/dataset/file_json.cc | 8 +- cpp/src/arrow/dataset/file_json.h | 5 +- cpp/src/arrow/dataset/file_parquet.cc | 358 +++++++++++++++------ cpp/src/arrow/dataset/file_parquet.h | 9 +- cpp/src/arrow/dataset/file_parquet_test.cc | 6 +- cpp/src/arrow/dataset/scan_node.cc | 2 +- cpp/src/arrow/dataset/scanner_test.cc | 2 +- cpp/src/arrow/dataset/test_util_internal.h | 53 ++- cpp/src/parquet/arrow/reader.cc | 1 - 15 files changed, 345 insertions(+), 145 deletions(-) diff --git a/cpp/src/arrow/dataset/dataset.cc b/cpp/src/arrow/dataset/dataset.cc index c611b297c3715..1970af5611d2a 100644 --- a/cpp/src/arrow/dataset/dataset.cc +++ b/cpp/src/arrow/dataset/dataset.cc @@ -68,7 +68,7 @@ Future> Fragment::InspectFragmentImpl( } Future> Fragment::BeginScan( - const FragmentScanRequest& request, const InspectedFragment& inspected_fragment, + const FragmentScanRequest& request, InspectedFragment* inspected_fragment, compute::ExecContext* exec_context) { return Status::NotImplemented("New scan method"); } @@ -202,7 +202,7 @@ class InMemoryFragment::Scanner : public FragmentScanner { }; Future> InMemoryFragment::BeginScan( - const FragmentScanRequest& request, const InspectedFragment& inspected_fragment, + const FragmentScanRequest& request, InspectedFragment* inspected_fragment, compute::ExecContext* exec_context) { return Future>::MakeFinished( std::make_shared(record_batches_)); diff --git a/cpp/src/arrow/dataset/dataset.h b/cpp/src/arrow/dataset/dataset.h index 48e57203454fd..80395060be42a 100644 --- a/cpp/src/arrow/dataset/dataset.h +++ b/cpp/src/arrow/dataset/dataset.h @@ -216,7 +216,7 @@ class ARROW_DS_EXPORT Fragment : public std::enable_shared_from_this { /// \brief Start a scan operation virtual Future> BeginScan( - const FragmentScanRequest& request, const InspectedFragment& inspected_fragment, + const FragmentScanRequest& request, InspectedFragment* inspected_fragment, compute::ExecContext* exec_context); /// \brief Count the number of rows in this fragment matching the filter using metadata @@ -292,7 +292,7 @@ class ARROW_DS_EXPORT InMemoryFragment : public Fragment { const FragmentScanOptions* format_options, compute::ExecContext* exec_context) override; Future> BeginScan( - const FragmentScanRequest& request, const InspectedFragment& inspected_fragment, + const FragmentScanRequest& request, InspectedFragment* inspected_fragment, compute::ExecContext* exec_context) override; std::string type_name() const override { return "in-memory"; } diff --git a/cpp/src/arrow/dataset/file_base.cc b/cpp/src/arrow/dataset/file_base.cc index e3e092052d3a9..dde66e4573efd 100644 --- a/cpp/src/arrow/dataset/file_base.cc +++ b/cpp/src/arrow/dataset/file_base.cc @@ -134,15 +134,15 @@ Future> FileFormat::CountRows( } Future> FileFormat::InspectFragment( - const FileSource& source, const FragmentScanOptions* format_options, - compute::ExecContext* exec_context) const { + const FileFragment& fragment, const FileSource& source, + const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const { return Status::NotImplemented("This format does not yet support the scan2 node"); } Future> FileFormat::BeginScan( const FileSource& source, const FragmentScanRequest& request, - const InspectedFragment& inspected_fragment, - const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const { + InspectedFragment* inspected_fragment, const FragmentScanOptions* format_options, + compute::ExecContext* exec_context) const { return Status::NotImplemented("This format does not yet support the scan2 node"); } @@ -181,11 +181,11 @@ Future> FileFragment::InspectFragmentImpl( if (format_options == nullptr) { realized_format_options = format_->default_fragment_scan_options.get(); } - return format_->InspectFragment(source_, realized_format_options, exec_context); + return format_->InspectFragment(*this, source_, realized_format_options, exec_context); } Future> FileFragment::BeginScan( - const FragmentScanRequest& request, const InspectedFragment& inspected_fragment, + const FragmentScanRequest& request, InspectedFragment* inspected_fragment, compute::ExecContext* exec_context) { const FragmentScanOptions* realized_format_options = request.format_scan_options; if (realized_format_options == nullptr) { diff --git a/cpp/src/arrow/dataset/file_base.h b/cpp/src/arrow/dataset/file_base.h index b46c8a3620693..730dc19711aa2 100644 --- a/cpp/src/arrow/dataset/file_base.h +++ b/cpp/src/arrow/dataset/file_base.h @@ -165,7 +165,8 @@ class ARROW_DS_EXPORT FileFormat : public std::enable_shared_from_this> InspectFragment( - const FileSource& source, const FragmentScanOptions* format_options, + const FileFragment& fragment, const FileSource& source, + const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const; virtual Result ScanBatchesAsync( @@ -182,8 +183,7 @@ class ARROW_DS_EXPORT FileFormat : public std::enable_shared_from_this> BeginScan( const FileSource& source, const FragmentScanRequest& request, - const InspectedFragment& inspected_fragment, - const FragmentScanOptions* format_options, + InspectedFragment* inspected_fragment, const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const; /// \brief Open a fragment @@ -226,7 +226,7 @@ class ARROW_DS_EXPORT FileFragment : public Fragment, compute::Expression predicate, const std::shared_ptr& options) override; Future> BeginScan( - const FragmentScanRequest& request, const InspectedFragment& inspected_fragment, + const FragmentScanRequest& request, InspectedFragment* inspected_fragment, compute::ExecContext* exec_context) override; Future> InspectFragmentImpl( const FragmentScanOptions* format_options, diff --git a/cpp/src/arrow/dataset/file_csv.cc b/cpp/src/arrow/dataset/file_csv.cc index 7e22987c794d3..9dd1ea44af6eb 100644 --- a/cpp/src/arrow/dataset/file_csv.cc +++ b/cpp/src/arrow/dataset/file_csv.cc @@ -401,11 +401,11 @@ Future> CsvFileFormat::CountRows( Future> CsvFileFormat::BeginScan( const FileSource& file_source, const FragmentScanRequest& request, - const InspectedFragment& inspected_fragment, - const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const { + InspectedFragment* inspected_fragment, const FragmentScanOptions* format_options, + compute::ExecContext* exec_context) const { auto csv_options = static_cast(format_options); - auto csv_fragment = static_cast(inspected_fragment); - return CsvFileScanner::Make(*csv_options, request, csv_fragment, + auto* csv_fragment = static_cast(inspected_fragment); + return CsvFileScanner::Make(*csv_options, request, *csv_fragment, exec_context->executor()); } @@ -432,8 +432,8 @@ Result> DoInspectFragment( } Future> CsvFileFormat::InspectFragment( - const FileSource& source, const FragmentScanOptions* format_options, - compute::ExecContext* exec_context) const { + const FileFragment& fragment, const FileSource& source, + const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const { ARROW_ASSIGN_OR_RAISE( const auto* csv_options, GetFragmentScanOptions(format_options, kCsvTypeName)); diff --git a/cpp/src/arrow/dataset/file_csv.h b/cpp/src/arrow/dataset/file_csv.h index a05a3cb479f71..8bea174735644 100644 --- a/cpp/src/arrow/dataset/file_csv.h +++ b/cpp/src/arrow/dataset/file_csv.h @@ -58,8 +58,7 @@ class ARROW_DS_EXPORT CsvFileFormat : public FileFormat { Future> BeginScan( const FileSource& file_source, const FragmentScanRequest& request, - const InspectedFragment& inspected_fragment, - const FragmentScanOptions* format_options, + InspectedFragment* inspected_fragment, const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const override; Result ScanBatchesAsync( @@ -67,7 +66,8 @@ class ARROW_DS_EXPORT CsvFileFormat : public FileFormat { const std::shared_ptr& file) const override; Future> InspectFragment( - const FileSource& source, const FragmentScanOptions* format_options, + const FileFragment& fragment, const FileSource& source, + const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const override; Future> CountRows( diff --git a/cpp/src/arrow/dataset/file_json.cc b/cpp/src/arrow/dataset/file_json.cc index aea5d6bdfb1dc..b776f1cea8290 100644 --- a/cpp/src/arrow/dataset/file_json.cc +++ b/cpp/src/arrow/dataset/file_json.cc @@ -401,8 +401,8 @@ Future> JsonFileFormat::CountRows( } Future> JsonFileFormat::InspectFragment( - const FileSource& source, const FragmentScanOptions* format_options, - compute::ExecContext* exec_context) const { + const FileFragment& fragment, const FileSource& source, + const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const { auto json_options = static_cast(format_options); auto* executor = source.filesystem() ? source.filesystem()->io_context().executor() : exec_context->executor(); @@ -414,11 +414,11 @@ Future> JsonFileFormat::InspectFragment( Future> JsonFileFormat::BeginScan( const FileSource& file_source, const FragmentScanRequest& scan_request, - const InspectedFragment& inspected, const FragmentScanOptions* format_options, + InspectedFragment* inspected, const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const { return JsonFragmentScanner::Make( scan_request, static_cast(*format_options), - static_cast(inspected), exec_context->executor()); + static_cast(*inspected), exec_context->executor()); } } // namespace dataset diff --git a/cpp/src/arrow/dataset/file_json.h b/cpp/src/arrow/dataset/file_json.h index d331cbdd98a44..28c77c6b8c2d4 100644 --- a/cpp/src/arrow/dataset/file_json.h +++ b/cpp/src/arrow/dataset/file_json.h @@ -54,12 +54,13 @@ class ARROW_DS_EXPORT JsonFileFormat : public FileFormat { Result> Inspect(const FileSource& source) const override; Future> InspectFragment( - const FileSource& source, const FragmentScanOptions* format_options, + const FileFragment& fragment, const FileSource& source, + const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const override; Future> BeginScan( const FileSource& file_source, const FragmentScanRequest& scan_request, - const InspectedFragment& inspected, const FragmentScanOptions* format_options, + InspectedFragment* inspected, const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const override; Result ScanBatchesAsync( diff --git a/cpp/src/arrow/dataset/file_parquet.cc b/cpp/src/arrow/dataset/file_parquet.cc index cf5ce3d5c010d..b09a912b5c68c 100644 --- a/cpp/src/arrow/dataset/file_parquet.cc +++ b/cpp/src/arrow/dataset/file_parquet.cc @@ -29,6 +29,7 @@ #include "arrow/acero/exec_plan.h" #include "arrow/compute/exec.h" +#include "arrow/compute/expression.h" #include "arrow/dataset/dataset.h" #include "arrow/dataset/dataset_internal.h" #include "arrow/dataset/scanner.h" @@ -37,7 +38,9 @@ #include "arrow/io/caching.h" #include "arrow/io/interfaces.h" #include "arrow/io/type_fwd.h" +#include "arrow/memory_pool.h" #include "arrow/table.h" +#include "arrow/type_fwd.h" #include "arrow/util/async_generator.h" #include "arrow/util/async_generator_fwd.h" #include "arrow/util/bit_util.h" @@ -70,6 +73,40 @@ using parquet::arrow::StatisticsAsScalars; namespace { +std::optional ColumnChunkStatisticsAsExpression( + const SchemaField& schema_field, const parquet::RowGroupMetaData& metadata) { + // For the remaining of this function, failure to extract/parse statistics + // are ignored by returning nullptr. The goal is two fold. First + // avoid an optimization which breaks the computation. Second, allow the + // following columns to maybe succeed in extracting column statistics. + + // For now, only leaf (primitive) types are supported. + if (!schema_field.is_leaf()) { + return std::nullopt; + } + + auto column_metadata = metadata.ColumnChunk(schema_field.column_index); + auto statistics = column_metadata->statistics(); + const auto& field = schema_field.field; + + if (statistics == nullptr) { + return std::nullopt; + } + + compute::Expression field_expr = compute::field_ref(field->name()); + + return ParquetFileFragment::EvaluateStatisticsAsExpression( + field_expr, schema_field.field->type(), *statistics); +} + +inline void FoldingAnd(compute::Expression* l, compute::Expression r) { + if (*l == compute::literal(true)) { + *l = std::move(r); + } else { + *l = and_(std::move(*l), std::move(r)); + } +} + Result> ColumnNamesFromManifest(const SchemaManifest& manifest) { std::vector names; names.reserve(manifest.schema_fields.size()); @@ -79,14 +116,165 @@ Result> ColumnNamesFromManifest(const SchemaManifest& m return names; } +Result ResolvePath(const FieldPath& path, + const SchemaManifest& manifest) { + const SchemaField* itr; + DCHECK_LT(path[0], static_cast(manifest.schema_fields.size())); + itr = &manifest.schema_fields[path[0]]; + for (std::size_t i = 1; i < path.indices().size(); i++) { + // This should be guaranteed by evolution but maybe we should be more flexible + // to account for bugs in the evolution strategy? + DCHECK(!itr->is_leaf()); + DCHECK_LT(path[i], static_cast(itr->children.size())); + itr = &itr->children[path[i]]; + } + return itr; +} + +// Opening a file reader async is a little bit of a nuisance since future's don't +// currently support moving the result and the file reader expects a unique_ptr to a +// parquet reader. This helper works around this problem with a bit of a hack. +Future> OpenFileReaderAsync( + std::shared_ptr file, + const parquet::ReaderProperties& properties, + std::function(const parquet::FileMetaData&)> + arrow_properties_factory, + MemoryPool* memory_pool, + std::shared_ptr file_metadata = nullptr) { + auto reader_fut = + parquet::ParquetFileReader::OpenAsync(std::move(file), properties, file_metadata); + return reader_fut.Then([reader_fut, arrow_properties_factory, memory_pool]( + const std::unique_ptr&) mutable + -> Result> { + ARROW_ASSIGN_OR_RAISE(std::unique_ptr reader, + reader_fut.MoveResult()); + std::unique_ptr arrow_reader; + ARROW_ASSIGN_OR_RAISE(parquet::ArrowReaderProperties arrow_properties, + arrow_properties_factory(*reader->metadata())); + RETURN_NOT_OK(parquet::arrow::FileReader::Make( + memory_pool, std::move(reader), std::move(arrow_properties), &arrow_reader)); + return arrow_reader; + }); +} + class ParquetInspectedFragment : public InspectedFragment { public: - explicit ParquetInspectedFragment(std::shared_ptr file_metadata, - std::vector column_names) + explicit ParquetInspectedFragment( + std::vector row_groups, std::shared_ptr file_metadata, + std::shared_ptr manifest, + std::shared_ptr schema, std::vector column_names) : InspectedFragment(std::move(column_names)), - file_metadata(std::move(file_metadata)) {} + row_groups_(std::move(row_groups)), + file_metadata_(std::move(file_metadata)), + manifest_(std::move(manifest)), + schema_(std::move(schema)), + statistics_expressions_(row_groups_.size(), compute::literal(true)) { + // "empty" means "all row groups" + if (row_groups_.empty()) { + row_groups_.resize(file_metadata_->num_row_groups()); + std::iota(row_groups_.begin(), row_groups_.end(), 0); + statistics_expressions_.resize(row_groups_.size(), compute::literal(true)); + } + // Note, this is only a lower bound, it could be higher if there were lots of nested + // refs but that is pretty unlikely + statistics_expressions_complete_.reserve(schema_->num_fields()); + } + + Result> TestRowGroups(compute::Expression predicate) { + if (!predicate.IsBound()) { + ARROW_ASSIGN_OR_RAISE(predicate, predicate.Bind(*schema_)); + } + + std::lock_guard lg(mutex_); + + if (!predicate.IsSatisfiable()) { + return std::vector{}; + } + + for (const FieldRef& ref : FieldsInExpression(predicate)) { + ARROW_ASSIGN_OR_RAISE(auto match, ref.FindOneOrNone(*schema_)); + + if (match.empty()) continue; + + ARROW_ASSIGN_OR_RAISE(const SchemaField* schema_field, + ResolvePath(match, *manifest_)); + if (!schema_field->is_leaf()) { + // Statistics are only kept for leaves + continue; + } + + if (statistics_expressions_complete_.find(match) != + statistics_expressions_complete_.end()) { + // We've already accounted for this field's statistics + continue; + } + statistics_expressions_complete_.insert(match); + + for (int row_group_idx = 0; row_group_idx < static_cast(row_groups_.size()); + row_group_idx++) { + int row_group = row_groups_[row_group_idx]; + auto row_group_metadata = file_metadata_->RowGroup(row_group); + std::shared_ptr col_stats = + row_group_metadata->ColumnChunk(schema_field->column_index)->statistics(); + if (col_stats == nullptr) { + continue; + } + + compute::Expression match_ref = compute::field_ref(match); + if (auto minmax = ParquetFileFragment::EvaluateStatisticsAsExpression( + match_ref, schema_field->field->type(), *col_stats)) { + FoldingAnd(&statistics_expressions_[row_group_idx], std::move(*minmax)); + ARROW_ASSIGN_OR_RAISE(statistics_expressions_[row_group_idx], + statistics_expressions_[row_group_idx].Bind(*schema_)); + } + } + } - std::shared_ptr file_metadata; + std::vector row_groups(row_groups_.size()); + for (size_t i = 0; i < row_groups_.size(); ++i) { + ARROW_ASSIGN_OR_RAISE(auto row_group_predicate, + SimplifyWithGuarantee(predicate, statistics_expressions_[i])); + row_groups[i] = std::move(row_group_predicate); + } + return row_groups; + } + + Result> FilterRowGroups(compute::Expression predicate) { + std::vector row_groups; + ARROW_ASSIGN_OR_RAISE(auto expressions, TestRowGroups(std::move(predicate))); + + std::lock_guard lg(mutex_); + DCHECK(expressions.empty() || (expressions.size() == row_groups_.size())); + for (size_t i = 0; i < expressions.size(); i++) { + if (expressions[i].IsSatisfiable()) { + row_groups.push_back(row_groups_.at(i)); + } + } + return row_groups; + } + + const std::shared_ptr& file_metadata() const { + return file_metadata_; + } + + private: + // The row groups that are included in this fragment + std::vector row_groups_; + // The parquet file metadata + std::shared_ptr file_metadata_; + // A helper utility that bridges parquet file metadata and the arrow schema + std::shared_ptr manifest_; + // The arrow schema of the parquet file (fragment schema) + std::shared_ptr schema_; + // Cached expressions computed from the row group statistics. One per row group. + std::vector statistics_expressions_; + // A cached flag that tells us whether we have calculated statistics for a given column + // or not. + std::unordered_set statistics_expressions_complete_; + // Guards concurrent access to statistics_expressions_ and + // statistics_expressions_complete_ which, being cached and lazily computed, may be + // updated during calls to Test/FilterRowGroups. + std::mutex mutex_; }; parquet::ReaderProperties MakeReaderProperties( @@ -144,29 +332,6 @@ bool IsNan(const Scalar& value) { return false; } -std::optional ColumnChunkStatisticsAsExpression( - const SchemaField& schema_field, const parquet::RowGroupMetaData& metadata) { - // For the remaining of this function, failure to extract/parse statistics - // are ignored by returning nullptr. The goal is two fold. First - // avoid an optimization which breaks the computation. Second, allow the - // following columns to maybe succeed in extracting column statistics. - - // For now, only leaf (primitive) types are supported. - if (!schema_field.is_leaf()) { - return std::nullopt; - } - - auto column_metadata = metadata.ColumnChunk(schema_field.column_index); - auto statistics = column_metadata->statistics(); - const auto& field = schema_field.field; - - if (statistics == nullptr) { - return std::nullopt; - } - - return ParquetFileFragment::EvaluateStatisticsAsExpression(*field, *statistics); -} - void AddColumnIndices(const SchemaField& schema_field, std::vector* column_projection) { if (schema_field.is_leaf()) { @@ -345,18 +510,20 @@ class ParquetFragmentScanner : public FragmentScanner { public: explicit ParquetFragmentScanner( std::string path, std::shared_ptr file_metadata, - const ParquetFragmentScanOptions* scan_options, + std::vector desired_row_groups, const ParquetFragmentScanOptions* scan_options, const ParquetFileFormat::ReaderOptions* format_reader_options, compute::ExecContext* exec_context) : path_(std::move(path)), file_metadata_(std::move(file_metadata)), + desired_row_groups_(std::move(desired_row_groups)), scan_options_(scan_options), format_reader_options_(format_reader_options), exec_context_(exec_context) {} AsyncGenerator> RunScanTask(int task_number) override { + int row_group_number = desired_row_groups_[task_number]; AsyncGenerator> row_group_batches = - file_reader_->ReadRowGroupAsync(task_number, desired_columns_, + file_reader_->ReadRowGroupAsync(row_group_number, desired_columns_, exec_context_->executor()); return MakeMappedGenerator( row_group_batches, @@ -376,13 +543,14 @@ class ParquetFragmentScanner : public FragmentScanner { std::move(arrays)); } - int NumScanTasks() override { return file_metadata_->num_row_groups(); } + int NumScanTasks() override { return static_cast(desired_row_groups_.size()); } int NumBatchesInScanTask(int task_number) override { + int row_group_number = desired_row_groups_[task_number]; if (scan_options_->allow_jumbo_values) { return FragmentScanner::kUnknownNumberOfBatches; } - int64_t num_rows = file_metadata_->RowGroup(task_number)->num_rows(); + int64_t num_rows = file_metadata_->RowGroup(row_group_number)->num_rows(); int64_t num_batches = bit_util::CeilDiv(num_rows, static_cast(batch_size_)); return static_cast(num_batches); } @@ -498,21 +666,6 @@ class ParquetFragmentScanner : public FragmentScanner { return Status::OK(); } - Result ResolvePath(const FieldPath& path, - const SchemaManifest& manifest) { - const SchemaField* itr; - DCHECK_LT(path[0], static_cast(manifest.schema_fields.size())); - itr = &manifest.schema_fields[path[0]]; - for (std::size_t i = 1; i < path.indices().size(); i++) { - // This should be guaranteed by evolution but maybe we should be more flexible - // to account for bugs in the evolution strategy? - DCHECK(!itr->is_leaf()); - DCHECK_LT(path[i], static_cast(itr->children.size())); - itr = &itr->children[path[i]]; - } - return itr; - } - void AddColumnIndices(const SchemaField& field, std::vector* indices) { if (field.is_leaf()) { indices->push_back(field.column_index); @@ -640,27 +793,26 @@ class ParquetFragmentScanner : public FragmentScanner { Future<> Initialize(std::shared_ptr file, const FragmentScanRequest& request) { parquet::ReaderProperties properties = MakeParquetReaderProperties(); - // TODO(ARROW-12259): workaround since we have Future<(move-only type)> - auto reader_fut = parquet::ParquetFileReader::OpenAsync( - std::move(file), std::move(properties), file_metadata_); - return reader_fut.Then( - [this, reader_fut, - request](const std::unique_ptr&) mutable -> Status { - ARROW_ASSIGN_OR_RAISE(std::unique_ptr reader, - reader_fut.MoveResult()); - std::shared_ptr metadata = reader->metadata(); - parquet::ArrowReaderProperties arrow_properties = - MakeParquetArrowReaderProperties(); - ARROW_RETURN_NOT_OK(CheckRowGroupSizes(arrow_properties.batch_size())); - if (batch_size_ > std::numeric_limits::max()) { - return Status::NotImplemented("Scanner batch size > int32_t max"); - } - batch_size_ = static_cast(arrow_properties.batch_size()); - RETURN_NOT_OK(parquet::arrow::FileReader::Make( - exec_context_->memory_pool(), std::move(reader), - std::move(arrow_properties), &file_reader_)); + auto arrow_properties_factory = + [this](const parquet::FileMetaData&) -> Result { + parquet::ArrowReaderProperties arrow_properties = + MakeParquetArrowReaderProperties(); + ARROW_RETURN_NOT_OK(CheckRowGroupSizes(arrow_properties.batch_size())); + batch_size_ = static_cast(arrow_properties.batch_size()); + if (batch_size_ > std::numeric_limits::max()) { + return Status::NotImplemented("Scanner batch size > int32_t max"); + } + return arrow_properties; + }; + Future> file_reader_fut = + OpenFileReaderAsync(std::move(file), properties, arrow_properties_factory, + exec_context_->memory_pool(), file_metadata_); + return file_reader_fut.Then( + [this, request]( + const std::shared_ptr& arrow_reader) -> Status { RETURN_NOT_OK(CalculateProjection(*file_metadata_, *request.fragment_selection, - file_reader_->manifest())); + arrow_reader->manifest())); + file_reader_ = arrow_reader; return Status::OK(); }, [this](const Status& status) -> Status { @@ -672,13 +824,15 @@ class ParquetFragmentScanner : public FragmentScanner { std::shared_ptr file, std::string path, const ParquetFragmentScanOptions* scan_options, const ParquetFileFormat::ReaderOptions* format_reader_options, - const FragmentScanRequest& request, const ParquetInspectedFragment& inspection, + const FragmentScanRequest& request, ParquetInspectedFragment* inspection, compute::ExecContext* exec_context) { + ARROW_ASSIGN_OR_RAISE(std::vector desired_row_groups, + inspection->FilterRowGroups(request.filter)); // Construct a fragment scanner, initialize it, and return it std::shared_ptr parquet_fragment_scanner = - std::make_shared(std::move(path), - inspection.file_metadata, scan_options, - format_reader_options, exec_context); + std::make_shared( + std::move(path), inspection->file_metadata(), std::move(desired_row_groups), + scan_options, format_reader_options, exec_context); return parquet_fragment_scanner->Initialize(std::move(file), request) .Then([fragment_scanner = std::static_pointer_cast( parquet_fragment_scanner)]() { return fragment_scanner; }); @@ -688,6 +842,7 @@ class ParquetFragmentScanner : public FragmentScanner { // These properties are set during construction std::string path_; std::shared_ptr file_metadata_; + std::vector desired_row_groups_; const ParquetFragmentScanOptions* scan_options_; const ParquetFileFormat::ReaderOptions* format_reader_options_; compute::ExecContext* exec_context_; @@ -695,16 +850,15 @@ class ParquetFragmentScanner : public FragmentScanner { // These are set during Initialize std::vector desired_columns_; std::vector selection_paths_; - std::unique_ptr file_reader_; + std::shared_ptr file_reader_; int32_t batch_size_; }; } // namespace std::optional ParquetFileFragment::EvaluateStatisticsAsExpression( - const Field& field, const parquet::Statistics& statistics) { - auto field_expr = compute::field_ref(field.name()); - + const compute::Expression& field_expr, const std::shared_ptr& field_type, + const parquet::Statistics& statistics) { // Optimize for corner case where all values are nulls if (statistics.num_values() == 0 && statistics.null_count() > 0) { return is_null(std::move(field_expr)); @@ -715,8 +869,8 @@ std::optional ParquetFileFragment::EvaluateStatisticsAsExpr return std::nullopt; } - auto maybe_min = min->CastTo(field.type()); - auto maybe_max = max->CastTo(field.type()); + auto maybe_min = min->CastTo(field_type); + auto maybe_max = max->CastTo(field_type); if (maybe_min.ok() && maybe_max.ok()) { min = maybe_min.MoveValueUnsafe(); @@ -800,38 +954,44 @@ Result> ParquetFileFormat::Inspect( } Future> ParquetFileFormat::InspectFragment( - const FileSource& source, const FragmentScanOptions* format_options, - compute::ExecContext* exec_context) const { + const FileFragment& fragment, const FileSource& source, + const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const { + const auto& parquet_fragment = checked_cast(fragment); ARROW_ASSIGN_OR_RAISE(const ParquetFragmentScanOptions* parquet_scan_options, GetFragmentScanOptions( format_options, kParquetTypeName)); auto properties = MakeReaderProperties(parquet_scan_options, exec_context->memory_pool()); + auto arrow_properties_factory = + [reader_opts = reader_options](const parquet::FileMetaData& file_metadata) { + return MakeArrowReaderProperties(reader_opts, file_metadata); + }; ARROW_ASSIGN_OR_RAISE(auto input, source.Open()); - auto reader_fut = - parquet::ParquetFileReader::OpenAsync(std::move(input), std::move(properties)); - return reader_fut.Then([this](const std::unique_ptr& reader) - -> Result> { - std::shared_ptr metadata = reader->metadata(); - parquet::ArrowReaderProperties arrow_reader_props = - MakeArrowReaderProperties(reader_options, *metadata); - ARROW_ASSIGN_OR_RAISE(std::shared_ptr manifest, - GetSchemaManifest(*metadata, arrow_reader_props)); - ARROW_ASSIGN_OR_RAISE(std::vector column_names, - ColumnNamesFromManifest(*manifest)); - return std::make_shared(std::move(metadata), - std::move(column_names)); - }); + return OpenFileReaderAsync(input, properties, arrow_properties_factory, + exec_context->memory_pool()) + .Then([row_groups = parquet_fragment.row_groups()]( + const std::shared_ptr& file_reader) mutable + -> Result> { + ARROW_ASSIGN_OR_RAISE(std::vector column_names, + ColumnNamesFromManifest(file_reader->manifest())); + std::shared_ptr file_schema; + ARROW_RETURN_NOT_OK(file_reader->GetSchema(&file_schema)); + auto manifest_copy = + std::make_shared(file_reader->manifest()); + return std::make_shared( + std::move(row_groups), file_reader->parquet_reader()->metadata(), + std::move(manifest_copy), file_schema, std::move(column_names)); + }); } Future> ParquetFileFormat::BeginScan( const FileSource& source, const FragmentScanRequest& request, - const InspectedFragment& inspected_fragment, - const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const { + InspectedFragment* inspected_fragment, const FragmentScanOptions* format_options, + compute::ExecContext* exec_context) const { ARROW_ASSIGN_OR_RAISE(const ParquetFragmentScanOptions* parquet_scan_options, GetFragmentScanOptions( format_options, kParquetTypeName)); - auto inspection = checked_cast(inspected_fragment); + auto* inspection = checked_cast(inspected_fragment); ARROW_ASSIGN_OR_RAISE(auto input, source.Open()); return ParquetFragmentScanner::Make(std::move(input), source.path(), parquet_scan_options, &reader_options, request, @@ -1186,14 +1346,6 @@ Result> ParquetFileFragment::Subset( return new_fragment; } -inline void FoldingAnd(compute::Expression* l, compute::Expression r) { - if (*l == compute::literal(true)) { - *l = std::move(r); - } else { - *l = and_(std::move(*l), std::move(r)); - } -} - Result> ParquetFileFragment::FilterRowGroups( compute::Expression predicate) { std::vector row_groups; diff --git a/cpp/src/arrow/dataset/file_parquet.h b/cpp/src/arrow/dataset/file_parquet.h index 08ad24c287cb5..b4217eba2aac8 100644 --- a/cpp/src/arrow/dataset/file_parquet.h +++ b/cpp/src/arrow/dataset/file_parquet.h @@ -96,13 +96,13 @@ class ARROW_DS_EXPORT ParquetFileFormat : public FileFormat { Result> Inspect(const FileSource& source) const override; Future> InspectFragment( - const FileSource& source, const FragmentScanOptions* format_options, + const FileFragment& fragment, const FileSource& source, + const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const override; Future> BeginScan( const FileSource& source, const FragmentScanRequest& request, - const InspectedFragment& inspected_fragment, - const FragmentScanOptions* format_options, + InspectedFragment* inspected_fragment, const FragmentScanOptions* format_options, compute::ExecContext* exec_context) const override; Result ScanBatchesAsync( @@ -182,7 +182,8 @@ class ARROW_DS_EXPORT ParquetFileFragment : public FileFragment { Result> Subset(std::vector row_group_ids); static std::optional EvaluateStatisticsAsExpression( - const Field& field, const parquet::Statistics& statistics); + const compute::Expression& field_expr, const std::shared_ptr& field_type, + const parquet::Statistics& statistics); private: ParquetFileFragment(FileSource source, std::shared_ptr format, diff --git a/cpp/src/arrow/dataset/file_parquet_test.cc b/cpp/src/arrow/dataset/file_parquet_test.cc index 7c8e3f5094c9e..b0111ab658efa 100644 --- a/cpp/src/arrow/dataset/file_parquet_test.cc +++ b/cpp/src/arrow/dataset/file_parquet_test.cc @@ -721,13 +721,14 @@ INSTANTIATE_TEST_SUITE_P(TestScan, TestParquetFileFormatScan, TEST(TestParquetStatistics, NullMax) { auto field = ::arrow::field("x", float32()); + auto field_ref = compute::field_ref("x"); ASSERT_OK_AND_ASSIGN(std::string dir_string, arrow::internal::GetEnvVar("PARQUET_TEST_DATA")); auto reader = parquet::ParquetFileReader::OpenFile(dir_string + "/nan_in_stats.parquet"); auto statistics = reader->RowGroup(0)->metadata()->ColumnChunk(0)->statistics(); - auto stat_expression = - ParquetFileFragment::EvaluateStatisticsAsExpression(*field, *statistics); + auto stat_expression = ParquetFileFragment::EvaluateStatisticsAsExpression( + field_ref, float32(), *statistics); EXPECT_EQ(stat_expression->ToString(), "(x >= 1)"); } @@ -747,6 +748,7 @@ TEST_P(TestParquetFileFormatScanNode, ScanSomeColumns) { TestScanSomeColumns(); TEST_P(TestParquetFileFormatScanNode, ScanSomeNestedColumns) { TestScanSomeNestedColumns(); } +TEST_P(TestParquetFileFormatScanNode, StatisticsFiltering) { TestStatisticsFiltering(); } TEST_P(TestParquetFileFormatScanNode, ScanWithInvalidOptions) { TestInvalidFormatScanOptions(); } diff --git a/cpp/src/arrow/dataset/scan_node.cc b/cpp/src/arrow/dataset/scan_node.cc index ec65cb28a3924..dc09014b310e1 100644 --- a/cpp/src/arrow/dataset/scan_node.cc +++ b/cpp/src/arrow/dataset/scan_node.cc @@ -605,7 +605,7 @@ class ScanNode : public acero::ExecNode, public acero::TracedNode { filter_minus_part, std::move(extracted.known_values))); return fragment_ - ->BeginScan(task_launcher->scan_request(), *inspected_fragment, + ->BeginScan(task_launcher->scan_request(), inspected_fragment.get(), node_->plan_->query_context()->exec_context()) .Then([this, task_launcher = std::move(task_launcher)]( const std::shared_ptr& fragment_scanner) mutable { diff --git a/cpp/src/arrow/dataset/scanner_test.cc b/cpp/src/arrow/dataset/scanner_test.cc index 0e33eb718a375..8129bfba6de2d 100644 --- a/cpp/src/arrow/dataset/scanner_test.cc +++ b/cpp/src/arrow/dataset/scanner_test.cc @@ -300,7 +300,7 @@ struct MockFragment : public Fragment { } Future> BeginScan( - const FragmentScanRequest& request, const InspectedFragment& inspected_fragment, + const FragmentScanRequest& request, InspectedFragment* inspected_fragment, compute::ExecContext* exec_context) override { has_started_ = true; seen_request_ = request; diff --git a/cpp/src/arrow/dataset/test_util_internal.h b/cpp/src/arrow/dataset/test_util_internal.h index 71bfc7ca6e030..f3c042b333908 100644 --- a/cpp/src/arrow/dataset/test_util_internal.h +++ b/cpp/src/arrow/dataset/test_util_internal.h @@ -1078,6 +1078,11 @@ class FileFormatFixtureMixinV2 : public ::testing::Test { SetScanProjectionAllColumns(); } + void SetDatasetSchema(std::shared_ptr schema) { + dataset_schema_ = std::move(schema); + SetScanProjectionAllColumns(); + } + void CheckDatasetSchemaSet() { DCHECK_NE(dataset_schema_, nullptr) << "call SetDatasetSchema before calling this method"; @@ -1286,6 +1291,14 @@ class FileFormatScanNodeMixin : public FileFormatFixtureMixinV2, GetParam().num_batches); } + Result> ReaderFromBatches( + std::vector> batches) { + std::shared_ptr schema = batches[0]->schema(); + Iterator> batch_it = + MakeVectorIterator(std::move(batches)); + return RecordBatchReader::MakeFromIterator(std::move(batch_it), schema); + } + // Scan the fragment through the scanner. Result> Scan(std::shared_ptr fragment, bool add_filter_fields = true) { @@ -1415,10 +1428,6 @@ class FileFormatScanNodeMixin : public FileFormatFixtureMixinV2, auto b = field("B", struct_({e, d})); this->SetDatasetSchema({a, b, c}); - // Only F - this->SetScanProjection({{1, 1, 0}}); - auto expected_schema = schema({f}); - auto CheckProjection = [&](std::vector projection, std::shared_ptr expected_schema) { this->SetScanProjection(projection); @@ -1441,6 +1450,42 @@ class FileFormatScanNodeMixin : public FileFormatFixtureMixinV2, CheckProjection({{0}, {1, 1, 0}, {1, 0}}, schema({a, f, e})); } + // Some formats support filtering out entire row groups based + // on row group statistics. This test case generates row groups + // that should be easily excluded and then makes sure the group + // is not included in the results. + void TestStatisticsFiltering() { + auto test_schema = schema({field("i32", int32()), field("f64", float64()), + field("struct", struct_({field("nested_i32", int32())}))}); + auto positive_values = RecordBatchFromJSON(test_schema, R"([ + [1, 1.0, { "nested_i32": 1 }], + [2, 2.0, { "nested_i32": 2 }], + [3, 3.0, { "nested_i32": 3 }] + ])"); + auto negative_values = RecordBatchFromJSON(test_schema, R"([ + [-1, -1.0, { "nested_i32": -1 }], + [-2, -2.0, { "nested_i32": -2 }] + ])"); + this->SetDatasetSchema(test_schema); + this->SetScanProjection({}); + + auto CheckFilter = [&](compute::Expression filter, int32_t num_expected_rows) { + this->SetScanFilter(std::move(filter)); + + ASSERT_OK_AND_ASSIGN(auto reader, + this->ReaderFromBatches({positive_values, negative_values})); + auto source = this->MakeBufferSource(reader.get()); + auto fragment = this->MakeFragment(*source); + ASSERT_OK_AND_ASSIGN(std::shared_ptr
table, ScanToTable(fragment)); + ASSERT_EQ(table->num_rows(), num_expected_rows); + }; + + CheckFilter(greater(field_ref("i32"), literal(0)), 3); + CheckFilter(less(field_ref("i32"), literal(0)), 2); + CheckFilter(greater(field_ref({2, 0}), literal(0)), 3); + CheckFilter(less(field_ref({2, 0}), literal(0)), 2); + } + void TestScanWithPushdownNulls() { // Regression test for ARROW-15312 auto i64 = field("i64", int64()); diff --git a/cpp/src/parquet/arrow/reader.cc b/cpp/src/parquet/arrow/reader.cc index aea05d92b3f20..44f1a74a53c2d 100644 --- a/cpp/src/parquet/arrow/reader.cc +++ b/cpp/src/parquet/arrow/reader.cc @@ -16,7 +16,6 @@ // under the License. #include "parquet/arrow/reader.h" -#include #include #include From f549145babcacc78733974db7520540df77e69c4 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 18 Jul 2023 11:15:22 -0700 Subject: [PATCH 09/13] Remove accidentally added header files --- cpp/src/arrow/dataset/file_parquet.cc | 2 -- 1 file changed, 2 deletions(-) diff --git a/cpp/src/arrow/dataset/file_parquet.cc b/cpp/src/arrow/dataset/file_parquet.cc index b09a912b5c68c..c5df518094370 100644 --- a/cpp/src/arrow/dataset/file_parquet.cc +++ b/cpp/src/arrow/dataset/file_parquet.cc @@ -50,14 +50,12 @@ #include "arrow/util/logging.h" #include "arrow/util/range.h" #include "arrow/util/tracing_internal.h" -#include "metadata.h" #include "parquet/arrow/reader.h" #include "parquet/arrow/schema.h" #include "parquet/arrow/writer.h" #include "parquet/file_reader.h" #include "parquet/properties.h" #include "parquet/statistics.h" -#include "schema.h" namespace arrow { From 8d2b2b9f89f79b5643f4b97b0ade25953bc875a0 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 18 Jul 2023 11:55:36 -0700 Subject: [PATCH 10/13] include fixes --- cpp/src/arrow/compute/key_map_avx2.cc | 2 -- cpp/src/arrow/dataset/file_parquet.cc | 13 ++++++-- cpp/src/arrow/dataset/file_parquet.h | 12 ++++---- cpp/src/arrow/dataset/scan_node.cc | 2 -- cpp/src/arrow/dataset/scanner_test.cc | 43 --------------------------- 5 files changed, 16 insertions(+), 56 deletions(-) diff --git a/cpp/src/arrow/compute/key_map_avx2.cc b/cpp/src/arrow/compute/key_map_avx2.cc index a71150a64e0bb..eb318ff188fbb 100644 --- a/cpp/src/arrow/compute/key_map_avx2.cc +++ b/cpp/src/arrow/compute/key_map_avx2.cc @@ -20,8 +20,6 @@ #include "arrow/compute/key_map.h" #include "arrow/util/logging.h" -#include "arrow/util/logging.h" - namespace arrow { namespace compute { diff --git a/cpp/src/arrow/dataset/file_parquet.cc b/cpp/src/arrow/dataset/file_parquet.cc index c5df518094370..1fd2720104c38 100644 --- a/cpp/src/arrow/dataset/file_parquet.cc +++ b/cpp/src/arrow/dataset/file_parquet.cc @@ -18,10 +18,12 @@ #include "arrow/dataset/file_parquet.h" #include +#include +#include #include #include #include -#include +#include #include #include #include @@ -38,9 +40,10 @@ #include "arrow/io/caching.h" #include "arrow/io/interfaces.h" #include "arrow/io/type_fwd.h" -#include "arrow/memory_pool.h" +#include "arrow/record_batch.h" +#include "arrow/scalar.h" #include "arrow/table.h" -#include "arrow/type_fwd.h" +#include "arrow/type.h" #include "arrow/util/async_generator.h" #include "arrow/util/async_generator_fwd.h" #include "arrow/util/bit_util.h" @@ -48,13 +51,17 @@ #include "arrow/util/future.h" #include "arrow/util/iterator.h" #include "arrow/util/logging.h" +#include "arrow/util/mutex.h" #include "arrow/util/range.h" +#include "arrow/util/thread_pool.h" #include "arrow/util/tracing_internal.h" #include "parquet/arrow/reader.h" #include "parquet/arrow/schema.h" #include "parquet/arrow/writer.h" #include "parquet/file_reader.h" +#include "parquet/metadata.h" #include "parquet/properties.h" +#include "parquet/schema.h" #include "parquet/statistics.h" namespace arrow { diff --git a/cpp/src/arrow/dataset/file_parquet.h b/cpp/src/arrow/dataset/file_parquet.h index b4217eba2aac8..f323ae4307f3b 100644 --- a/cpp/src/arrow/dataset/file_parquet.h +++ b/cpp/src/arrow/dataset/file_parquet.h @@ -26,20 +26,20 @@ #include #include +#include "arrow/compute/expression.h" +#include "arrow/dataset/dataset.h" #include "arrow/dataset/discovery.h" #include "arrow/dataset/file_base.h" +#include "arrow/dataset/partition.h" #include "arrow/dataset/type_fwd.h" #include "arrow/dataset/visibility.h" -#include "arrow/io/caching.h" +#include "arrow/filesystem/type_fwd.h" +#include "arrow/type_fwd.h" +#include "arrow/util/future.h" namespace parquet { -class ParquetFileReader; class Statistics; -class ColumnChunkMetaData; -class RowGroupMetaData; class FileMetaData; -class FileDecryptionProperties; -class FileEncryptionProperties; class ReaderProperties; class ArrowReaderProperties; diff --git a/cpp/src/arrow/dataset/scan_node.cc b/cpp/src/arrow/dataset/scan_node.cc index dc09014b310e1..2d4378d137b57 100644 --- a/cpp/src/arrow/dataset/scan_node.cc +++ b/cpp/src/arrow/dataset/scan_node.cc @@ -16,7 +16,6 @@ // under the License. #include -#include #include #include #include @@ -43,7 +42,6 @@ #include "arrow/util/unreachable.h" using namespace std::string_view_literals; // NOLINT -using namespace std::placeholders; // NOLINT namespace arrow { diff --git a/cpp/src/arrow/dataset/scanner_test.cc b/cpp/src/arrow/dataset/scanner_test.cc index 8129bfba6de2d..f503296ad1815 100644 --- a/cpp/src/arrow/dataset/scanner_test.cc +++ b/cpp/src/arrow/dataset/scanner_test.cc @@ -810,49 +810,6 @@ TEST(TestNewScanner, OutOfOrderFragmentCompletion) { ASSERT_EQ(3, batches.size()); } -// TEST(TestNewScanner, OutOfOrderScanTaskCompletion) { -// constexpr int kNumFragments = 1; -// constexpr int kNumScanTasksPerFragment = 4; -// constexpr int kNumBatchesPerScanTask = 1; - -// internal::Initialize(); -// std::shared_ptr test_dataset = -// MakeTestDataset(kNumFragments, kNumScanTasksPerFragment, kNumBatchesPerScanTask); - -// ScanV2Options options(test_dataset); -// options.columns = ScanV2Options::AllColumns(*test_dataset->schema()); - -// // Begin scan -// acero::Declaration scan_decl = acero::Declaration("scan2", std::move(options)); -// Future batches_fut = -// acero::DeclarationToBatchesAsync(std::move(scan_decl)); - -// // Start scanning on all fragments -// for (int i = 0; i < kNumFragments; i++) { -// test_dataset->fragments_[i]->FinishInspection(); -// test_dataset->fragments_[i]->FinishScanBegin(); -// } - -// // Let scan tasks get started -// SleepABit(); -// // A fragment in the middle finishes -// test_dataset->fragments_[1]->fragment_scanner_->scan_tasks_[0].Finish(0); - -// SleepABit(); - -// // A fragment at the end finishes -// test_dataset->fragments_[2]->fragment_scanner_->scan_tasks_[0].Finish(0); - -// SleepABit(); - -// // Now the first fragment finishes -// test_dataset->fragments_[0]->fragment_scanner_->scan_tasks_[0].Finish(0); - -// // The scan should finish cleanly -// ASSERT_FINISHES_OK_AND_ASSIGN(RecordBatchVector batches, batches_fut); -// ASSERT_EQ(3, batches.size()); -// } - TEST(TestNewScanner, NestedRead) { // This tests the case where the file format does not support // handling nested reads (e.g. JSON) and so the scanner must From 5999301c258e671adf27c873eb9ca6521122c71d Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 18 Jul 2023 12:59:02 -0700 Subject: [PATCH 11/13] Fixing bad include --- cpp/src/arrow/dataset/test_util_internal.h | 1 - 1 file changed, 1 deletion(-) diff --git a/cpp/src/arrow/dataset/test_util_internal.h b/cpp/src/arrow/dataset/test_util_internal.h index f3c042b333908..fd6dedffb19c0 100644 --- a/cpp/src/arrow/dataset/test_util_internal.h +++ b/cpp/src/arrow/dataset/test_util_internal.h @@ -61,7 +61,6 @@ #include "arrow/util/pcg_random.h" #include "arrow/util/thread_pool.h" #include "arrow/util/vector.h" -#include "gmock/gmock.h" namespace arrow { From b6ff36dd6b0509f076fb9c4ab68701e56c17fdfa Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 18 Jul 2023 14:25:08 -0700 Subject: [PATCH 12/13] Minor lint fixes --- cpp/src/arrow/dataset/dataset.cc | 2 +- cpp/src/arrow/dataset/scanner_test.cc | 2 +- cpp/src/parquet/arrow/reader.h | 17 ++++++++++++++--- 3 files changed, 16 insertions(+), 5 deletions(-) diff --git a/cpp/src/arrow/dataset/dataset.cc b/cpp/src/arrow/dataset/dataset.cc index 1970af5611d2a..bffee17968275 100644 --- a/cpp/src/arrow/dataset/dataset.cc +++ b/cpp/src/arrow/dataset/dataset.cc @@ -193,7 +193,7 @@ class InMemoryFragment::Scanner : public FragmentScanner { int NumScanTasks() override { return 1; } int NumBatchesInScanTask(int task_number) override { - DCHECK_LE(batches_.size(), std::numeric_limits::max()); + DCHECK_LE(batches_.size(), static_cast(std::numeric_limits::max())); return static_cast(batches_.size()); } diff --git a/cpp/src/arrow/dataset/scanner_test.cc b/cpp/src/arrow/dataset/scanner_test.cc index f503296ad1815..1766f16357f2c 100644 --- a/cpp/src/arrow/dataset/scanner_test.cc +++ b/cpp/src/arrow/dataset/scanner_test.cc @@ -686,7 +686,7 @@ void CheckScannerBackpressure(std::shared_ptr dataset, ScanV2Option int total_scan_tasks = 0; for (const auto& frag : dataset->fragments_) { - total_scan_tasks += frag->fragment_scanner_->scan_tasks_.size(); + total_scan_tasks += static_cast(frag->fragment_scanner_->scan_tasks_.size()); frag->FinishInspection(); frag->FinishScanBegin(); } diff --git a/cpp/src/parquet/arrow/reader.h b/cpp/src/parquet/arrow/reader.h index 8442f9fb25a44..2a9c1ceec736a 100644 --- a/cpp/src/parquet/arrow/reader.h +++ b/cpp/src/parquet/arrow/reader.h @@ -265,6 +265,9 @@ class PARQUET_EXPORT FileReader { /// /// \param i the index of the row group to read /// \param cpu_executor an executor to use to run CPU tasks + /// \param allow_sliced_batches if false, an error is raised if a batch has too much + /// data for the given batch size. If true, smaller + /// batches will be returned instead. virtual AsyncBatchGenerator ReadRowGroupAsync(int i, ::arrow::internal::Executor* cpu_executor, bool allow_sliced_batches = false) = 0; @@ -276,6 +279,9 @@ class PARQUET_EXPORT FileReader { /// \param i the index of the row group to read /// \param column_indices leaf-indices of the columns to read /// \param cpu_executor an executor to use to run CPU tasks + /// \param allow_sliced_batches if false, an error is raised if a batch has too much + /// data for the given batch size. If true, smaller + /// batches will be returned instead. virtual AsyncBatchGenerator ReadRowGroupAsync(int i, const std::vector& column_indices, ::arrow::internal::Executor* cpu_executor, @@ -287,6 +293,9 @@ class PARQUET_EXPORT FileReader { /// /// \param row_groups indices of the row groups to read /// \param cpu_executor an executor to use to run CPU tasks + /// \param allow_sliced_batches if false, an error is raised if a batch has too much + /// data for the given batch size. If true, smaller + /// batches will be returned instead. virtual AsyncBatchGenerator ReadRowGroupsAsync( const std::vector& row_groups, ::arrow::internal::Executor* cpu_executor, bool allow_sliced_batches = false) = 0; @@ -308,15 +317,17 @@ class PARQUET_EXPORT FileReader { /// smaller. This can happen, for example, when there is not enough data or when a /// string column is too large to fit into a single batch. The parameter /// `allow_sliced_batches` can be set to false to disallow this later case. This can be - /// useful when you need to know exactly how many batches you will get from a scan - /// before you start. + /// useful when you need to know exactly how many batches you will get from the + /// operation before you start. /// /// The I/O executor is obtained from the I/O context in the reader properties. /// /// \param row_groups indices of the row groups to read /// \param column_indices indices of the columns to read /// \param cpu_executor an executor to use to run CPU tasks - /// \param allow_sliced_batches indicates whether or not we can slice large batches + /// \param allow_sliced_batches if false, an error is raised if a batch has too much + /// data for the given batch size. If false, smaller + /// batches will be returned instead. virtual AsyncBatchGenerator ReadRowGroupsAsync( const std::vector& row_groups, const std::vector& column_indices, ::arrow::internal::Executor* cpu_executor, bool allow_sliced_batches = false) = 0; From a667c1d2a7cf8850fea93a561e17790bcc534c9a Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 18 Jul 2023 14:50:42 -0700 Subject: [PATCH 13/13] Still working on lint errors --- cpp/src/arrow/dataset/dataset.cc | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/dataset/dataset.cc b/cpp/src/arrow/dataset/dataset.cc index bffee17968275..f119829a7ccc3 100644 --- a/cpp/src/arrow/dataset/dataset.cc +++ b/cpp/src/arrow/dataset/dataset.cc @@ -193,7 +193,8 @@ class InMemoryFragment::Scanner : public FragmentScanner { int NumScanTasks() override { return 1; } int NumBatchesInScanTask(int task_number) override { - DCHECK_LE(batches_.size(), static_cast(std::numeric_limits::max())); + DCHECK_LE(batches_.size(), + static_cast(std::numeric_limits::max())); return static_cast(batches_.size()); }