From 96c3bf9ffbf3fff11f81b0d8579788471c9fbb03 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Thu, 19 Aug 2021 17:52:01 -1000 Subject: [PATCH 01/26] ARROW-13542: Created dataset writer --- cpp/src/arrow/CMakeLists.txt | 1 + cpp/src/arrow/dataset/CMakeLists.txt | 2 + cpp/src/arrow/dataset/dataset_writer.cc | 569 ++++++++++++++++++ cpp/src/arrow/dataset/dataset_writer.h | 95 +++ cpp/src/arrow/dataset/dataset_writer_test.cc | 325 ++++++++++ cpp/src/arrow/dataset/file_base.cc | 267 ++------ cpp/src/arrow/dataset/file_base.h | 24 + cpp/src/arrow/dataset/file_ipc.cc | 3 - cpp/src/arrow/dataset/file_ipc_test.cc | 4 +- cpp/src/arrow/dataset/file_test.cc | 1 + cpp/src/arrow/dataset/test_util.h | 13 +- cpp/src/arrow/filesystem/filesystem.cc | 13 +- cpp/src/arrow/filesystem/filesystem.h | 5 +- cpp/src/arrow/filesystem/hdfs_test.cc | 9 +- cpp/src/arrow/filesystem/test_util.cc | 38 +- cpp/src/arrow/filesystem/test_util.h | 19 + cpp/src/arrow/testing/gtest_util.cc | 20 +- cpp/src/arrow/testing/gtest_util.h | 9 +- cpp/src/arrow/util/CMakeLists.txt | 1 + cpp/src/arrow/util/counting_semaphore.cc | 126 ++++ cpp/src/arrow/util/counting_semaphore.h | 60 ++ cpp/src/arrow/util/counting_semaphore_test.cc | 98 +++ cpp/src/jni/dataset/jni_wrapper.cc | 2 +- 23 files changed, 1463 insertions(+), 241 deletions(-) create mode 100644 cpp/src/arrow/dataset/dataset_writer.cc create mode 100644 cpp/src/arrow/dataset/dataset_writer.h create mode 100644 cpp/src/arrow/dataset/dataset_writer_test.cc create mode 100644 cpp/src/arrow/util/counting_semaphore.cc create mode 100644 cpp/src/arrow/util/counting_semaphore.h create mode 100644 cpp/src/arrow/util/counting_semaphore_test.cc diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index 70119df9e46..03cdb7f4902 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -201,6 +201,7 @@ set(ARROW_SRCS util/bpacking.cc util/cancel.cc util/compression.cc + util/counting_semaphore.cc util/cpu_info.cc util/decimal.cc util/delimiting.cc diff --git a/cpp/src/arrow/dataset/CMakeLists.txt b/cpp/src/arrow/dataset/CMakeLists.txt index 33f27fcda14..658eb0f9172 100644 --- a/cpp/src/arrow/dataset/CMakeLists.txt +++ b/cpp/src/arrow/dataset/CMakeLists.txt @@ -21,6 +21,7 @@ arrow_install_all_headers("arrow/dataset") set(ARROW_DATASET_SRCS dataset.cc + dataset_writer.cc discovery.cc file_base.cc file_ipc.cc @@ -110,6 +111,7 @@ function(ADD_ARROW_DATASET_TEST REL_TEST_NAME) endfunction() add_arrow_dataset_test(dataset_test) +add_arrow_dataset_test(dataset_writer_test) add_arrow_dataset_test(discovery_test) add_arrow_dataset_test(file_ipc_test) add_arrow_dataset_test(file_test) diff --git a/cpp/src/arrow/dataset/dataset_writer.cc b/cpp/src/arrow/dataset/dataset_writer.cc new file mode 100644 index 00000000000..0b47017a3e4 --- /dev/null +++ b/cpp/src/arrow/dataset/dataset_writer.cc @@ -0,0 +1,569 @@ +// 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 "arrow/dataset/dataset_writer.h" + +#include +#include +#include + +#include "arrow/filesystem/path_util.h" +#include "arrow/record_batch.h" +#include "arrow/result.h" +#include "arrow/util/future.h" +#include "arrow/util/logging.h" +#include "arrow/util/map.h" +#include "arrow/util/string.h" + +namespace arrow { +namespace dataset { + +namespace { + +constexpr util::string_view kIntegerToken = "{i}"; + +class DatasetWriterStatistics { + public: + DatasetWriterStatistics(uint64_t max_rows_in_flight, uint32_t max_files_in_flight) + : max_rows_in_flight_(max_rows_in_flight), + max_files_in_flight_(max_files_in_flight) {} + + bool CanWrite(const std::shared_ptr& record_batch, + const std::string& filename) { + std::lock_guard lg(mutex_); + uint64_t rows = record_batch->num_rows(); + DCHECK_LT(rows, max_rows_in_flight_); + + if (rows_in_flight_ + rows > max_rows_in_flight_) { + rows_in_waiting_ = rows; + backpressure_ = Future<>::Make(); + return false; + } + return true; + } + + bool CanOpenFile() { + std::lock_guard lg(mutex_); + if (files_in_flight_ == max_files_in_flight_) { + waiting_on_file_ = true; + backpressure_ = Future<>::Make(); + return false; + } + return true; + } + + void RecordWriteStart(uint64_t num_rows) { + std::lock_guard lg(mutex_); + rows_in_flight_ += num_rows; + } + + void RecordFileStart() { + std::lock_guard lg(mutex_); + files_in_flight_++; + } + + void RecordFileFinished() { + std::unique_lock lk(mutex_); + files_in_flight_--; + FreeBackpressureIfPossible(std::move(lk)); + } + + void RecordWriteFinish(uint64_t num_rows) { + std::unique_lock lk(mutex_); + rows_in_flight_ -= num_rows; + FreeBackpressureIfPossible(std::move(lk)); + } + + Future<> backpressure() { + std::lock_guard lg(mutex_); + return backpressure_; + } + + private: + void FreeBackpressureIfPossible(std::unique_lock&& lk) { + if (waiting_on_file_) { + if (files_in_flight_ < max_files_in_flight_) { + waiting_on_file_ = false; + } + } + + bool waiting_on_rows = true; + if (rows_in_flight_ > 0) { + if (rows_in_flight_ + rows_in_waiting_ < max_rows_in_flight_) { + rows_in_waiting_ = 0; + waiting_on_rows = false; + } + } else { + waiting_on_rows = false; + } + + if (backpressure_.is_valid() && !waiting_on_rows && !waiting_on_file_) { + Future<> old_backpressure = backpressure_; + backpressure_ = Future<>(); + lk.unlock(); + old_backpressure.MarkFinished(); + } + } + + uint64_t max_rows_in_flight_; + uint32_t max_files_in_flight_; + + Future<> backpressure_; + uint64_t rows_in_flight_ = 0; + uint64_t rows_in_waiting_ = 0; + uint32_t files_in_flight_ = 0; + bool waiting_on_file_ = false; + std::mutex mutex_; +}; + +class DatasetWriterFileQueue : public util::AsyncCloseable { + public: + explicit DatasetWriterFileQueue(const Future>& writer_fut, + const FileSystemDatasetWriteOptions& options, + std::mutex* visitors_mutex) + : options_(options), visitors_mutex_(visitors_mutex) { + running_task_ = Future<>::Make(); + writer_fut.AddCallback( + [this](const Result>& maybe_writer) { + if (maybe_writer.ok()) { + writer_ = *maybe_writer; + Flush(); + } else { + Abort(maybe_writer.status()); + } + }); + } + + Future Push(std::shared_ptr batch) { + std::unique_lock lk(mutex); + write_queue_.push_back(std::move(batch)); + Future write_future = Future::Make(); + write_futures_.push_back(write_future); + if (!running_task_.is_valid()) { + running_task_ = Future<>::Make(); + FlushUnlocked(std::move(lk)); + } + return write_future; + } + + Future<> DoClose() override { + std::lock_guard lg(mutex); + if (!running_task_.is_valid()) { + RETURN_NOT_OK(DoFinish()); + return Future<>::MakeFinished(); + } + return running_task_.Then([this] { return DoFinish(); }); + } + + private: + Future WriteNext() { + // May want to prototype / measure someday pushing the async write down further + return DeferNotOk( + io::default_io_context().executor()->Submit([this]() -> Result { + DCHECK(running_task_.is_valid()); + std::unique_lock lk(mutex); + const std::shared_ptr& to_write = write_queue_.front(); + Future on_complete = write_futures_.front(); + uint64_t rows_to_write = to_write->num_rows(); + lk.unlock(); + Status status = writer_->Write(to_write); + lk.lock(); + write_queue_.pop_front(); + write_futures_.pop_front(); + lk.unlock(); + if (!status.ok()) { + on_complete.MarkFinished(status); + } else { + on_complete.MarkFinished(rows_to_write); + } + return rows_to_write; + })); + } + + Status DoFinish() { + { + std::lock_guard lg(*visitors_mutex_); + RETURN_NOT_OK(options_.writer_pre_finish(writer_.get())); + } + RETURN_NOT_OK(writer_->Finish()); + { + std::lock_guard lg(*visitors_mutex_); + return options_.writer_post_finish(writer_.get()); + } + } + + void Abort(Status err) { + std::vector> futures_to_abort; + Future<> old_running_task = running_task_; + { + std::lock_guard lg(mutex); + write_queue_.clear(); + futures_to_abort = + std::vector>(write_futures_.begin(), write_futures_.end()); + write_futures_.clear(); + running_task_ = Future<>(); + } + for (auto& fut : futures_to_abort) { + fut.MarkFinished(err); + } + old_running_task.MarkFinished(std::move(err)); + } + + void Flush() { + std::unique_lock lk(mutex); + FlushUnlocked(std::move(lk)); + } + + void FlushUnlocked(std::unique_lock lk) { + if (write_queue_.empty()) { + Future<> old_running_task = running_task_; + running_task_ = Future<>(); + lk.unlock(); + old_running_task.MarkFinished(); + return; + } + WriteNext().AddCallback([this](const Result& res) { + if (res.ok()) { + Flush(); + } else { + Abort(res.status()); + } + }); + } + + const FileSystemDatasetWriteOptions& options_; + std::mutex* visitors_mutex_; + std::shared_ptr writer_; + std::mutex mutex; + std::list> write_queue_; + std::list> write_futures_; + Future<> running_task_; +}; + +struct WriteTask { + std::string filename; + uint64_t num_rows; +}; + +class DatasetWriterDirectoryQueue : public util::AsyncCloseable { + public: + DatasetWriterDirectoryQueue(util::AsyncCloseable* parent, std::string directory, + std::shared_ptr schema, + const FileSystemDatasetWriteOptions& write_options, + DatasetWriterStatistics* write_statistics, + std::mutex* visitors_mutex) + : util::AsyncCloseable(parent), + directory_(std::move(directory)), + schema_(std::move(schema)), + write_options_(write_options), + write_statistics_(write_statistics), + visitors_mutex_(visitors_mutex) {} + + Result> NextWritableChunk( + std::shared_ptr batch, std::shared_ptr* remainder, + bool* will_open_file) { + RETURN_NOT_OK(CheckClosed()); + DCHECK_GT(batch->num_rows(), 0); + uint64_t rows_available = std::numeric_limits::max(); + *will_open_file = rows_written_ == 0; + if (write_options_.max_rows_per_file > 0) { + rows_available = write_options_.max_rows_per_file - rows_written_; + } + + std::shared_ptr to_queue; + if (rows_available < static_cast(batch->num_rows())) { + to_queue = batch->Slice(0, static_cast(rows_available)); + *remainder = batch->Slice(static_cast(rows_available)); + } else { + to_queue = std::move(batch); + } + return to_queue; + } + + Future StartWrite(const std::shared_ptr& batch) { + RETURN_NOT_OK(CheckClosed()); + rows_written_ += batch->num_rows(); + WriteTask task{current_filename_, static_cast(batch->num_rows())}; + if (!latest_open_file_) { + latest_open_file_ = OpenFileQueue(current_filename_); + } + return latest_open_file_->Push(batch).Then([task] { return task; }); + } + + Result GetNextFilename() { + auto basename = ::arrow::internal::Replace( + write_options_.basename_template, kIntegerToken, std::to_string(file_counter_++)); + if (!basename) { + return Status::Invalid("string interpolation of basename template failed"); + } + + return fs::internal::ConcatAbstractPath(directory_, *basename); + } + + Status FinishCurrentFile() { + if (latest_open_file_) { + latest_open_file_ = nullptr; + } + rows_written_ = 0; + return GetNextFilename().Value(¤t_filename_); + } + + Result> OpenWriter(const std::string& filename) { + ARROW_ASSIGN_OR_RAISE(std::shared_ptr out_stream, + write_options_.filesystem->OpenOutputStream(filename)); + return write_options_.format()->MakeWriter(std::move(out_stream), schema_, + write_options_.file_write_options, + {write_options_.filesystem, filename}); + } + + std::shared_ptr OpenFileQueue(const std::string& filename) { + write_statistics_->RecordFileStart(); + Future> file_writer_fut = + init_future_.Then([this, filename] { + ::arrow::internal::Executor* io_executor = + write_options_.filesystem->io_context().executor(); + return DeferNotOk( + io_executor->Submit([this, filename]() { return OpenWriter(filename); })); + }); + auto file_queue = nursery_->MakeSharedCloseable( + file_writer_fut, write_options_, visitors_mutex_); + AddDependentTask( + file_queue->OnClosed().Then([this] { write_statistics_->RecordFileFinished(); })); + return file_queue; + } + + const std::string& current_filename() const { return current_filename_; } + uint64_t rows_written() const { return rows_written_; } + + void PrepareDirectory() { + init_future_ = + DeferNotOk(write_options_.filesystem->io_context().executor()->Submit([this] { + RETURN_NOT_OK(write_options_.filesystem->CreateDir(directory_)); + if (write_options_.existing_data_behavior == kDeleteMatchingPartitions) { + fs::FileSelector selector; + selector.base_dir = directory_; + selector.recursive = true; + return write_options_.filesystem->DeleteFiles(selector); + } + return Status::OK(); + })); + } + + Future<> DoClose() override { return FinishCurrentFile(); } + + static Result> Make( + util::AsyncCloseable* parent, util::Nursery* nursery, + const FileSystemDatasetWriteOptions& write_options, + DatasetWriterStatistics* write_statistics, std::shared_ptr schema, + std::string dir, std::mutex* visitors_mutex) { + auto dir_queue = nursery->MakeSharedCloseable( + parent, std::move(dir), std::move(schema), write_options, write_statistics, + visitors_mutex); + dir_queue->PrepareDirectory(); + ARROW_ASSIGN_OR_RAISE(dir_queue->current_filename_, dir_queue->GetNextFilename()); + return dir_queue; + } + + private: + std::string directory_; + std::shared_ptr schema_; + const FileSystemDatasetWriteOptions& write_options_; + DatasetWriterStatistics* write_statistics_; + std::mutex* visitors_mutex_; + Future<> init_future_; + std::string current_filename_; + std::shared_ptr latest_open_file_; + uint64_t rows_written_ = 0; + uint32_t file_counter_ = 0; +}; + +Status ValidateBasenameTemplate(util::string_view basename_template) { + if (basename_template.find(fs::internal::kSep) != util::string_view::npos) { + return Status::Invalid("basename_template contained '/'"); + } + size_t token_start = basename_template.find(kIntegerToken); + if (token_start == util::string_view::npos) { + return Status::Invalid("basename_template did not contain '", kIntegerToken, "'"); + } + return Status::OK(); +} + +Status EnsureDestinationValid(const FileSystemDatasetWriteOptions& options) { + if (options.existing_data_behavior == kError) { + fs::FileSelector selector; + selector.base_dir = options.base_dir; + selector.recursive = true; + Result> maybe_files = + options.filesystem->GetFileInfo(selector); + if (!maybe_files.ok()) { + // If the path doesn't exist then continue + return Status::OK(); + } + if (maybe_files->size() > 1) { + return Status::Invalid( + "Could not write to ", options.base_dir, + " as the directory is not empty and existing_data_behavior is kError"); + } + } + return Status::OK(); +} + +} // namespace + +class DatasetWriter::DatasetWriterImpl : public util::AsyncCloseable { + public: + DatasetWriterImpl(FileSystemDatasetWriteOptions write_options, uint64_t max_rows_queued) + : write_options_(std::move(write_options)), + statistics_(max_rows_queued, write_options.max_open_files) {} + + Future<> WriteRecordBatch(std::shared_ptr batch, + const std::string& directory) { + RETURN_NOT_OK(CheckClosed()); + RETURN_NOT_OK(CheckError()); + if (!directory.empty()) { + auto full_path = + fs::internal::ConcatAbstractPath(write_options_.base_dir, directory); + return DoWriteRecordBatch(std::move(batch), full_path); + } else { + return DoWriteRecordBatch(std::move(batch), write_options_.base_dir); + } + } + + Future<> DoClose() override { + directory_queues_.clear(); + return CheckError(); + } + + protected: + Status CloseLargestFile() { + std::shared_ptr largest = nullptr; + uint64_t largest_num_rows = 0; + for (auto& dir_queue : directory_queues_) { + if (dir_queue.second->rows_written() > largest_num_rows) { + largest_num_rows = dir_queue.second->rows_written(); + largest = dir_queue.second; + } + } + DCHECK_NE(largest, nullptr); + return largest->FinishCurrentFile(); + } + + Future<> DoWriteRecordBatch(std::shared_ptr batch, + const std::string& directory) { + ARROW_ASSIGN_OR_RAISE( + auto dir_queue_itr, + ::arrow::internal::GetOrInsertGenerated( + &directory_queues_, directory, [this, &batch](const std::string& dir) { + return DatasetWriterDirectoryQueue::Make(this, nursery_, write_options_, + &statistics_, batch->schema(), dir, + &visitors_mutex_); + })); + std::shared_ptr dir_queue = dir_queue_itr->second; + std::vector> scheduled_writes; + bool hit_backpressure = false; + while (batch) { + // Keep opening new files until batch is done. + std::shared_ptr remainder; + bool will_open_file = false; + ARROW_ASSIGN_OR_RAISE(auto next_chunk, dir_queue->NextWritableChunk( + batch, &remainder, &will_open_file)); + + if (statistics_.CanWrite(next_chunk, dir_queue->current_filename()) && + (!will_open_file || statistics_.CanOpenFile())) { + statistics_.RecordWriteStart(next_chunk->num_rows()); + scheduled_writes.push_back(dir_queue->StartWrite(next_chunk)); + batch = std::move(remainder); + if (batch) { + ARROW_RETURN_NOT_OK(dir_queue->FinishCurrentFile()); + } + } else { + if (!statistics_.CanOpenFile()) { + ARROW_RETURN_NOT_OK(CloseLargestFile()); + } + hit_backpressure = true; + break; + } + } + + for (auto& scheduled_write : scheduled_writes) { + // One of the below callbacks could run immediately and set err_ so we check + // it each time through the loop + RETURN_NOT_OK(CheckError()); + AddDependentTask(scheduled_write.Then( + [this](const WriteTask& write) { + statistics_.RecordWriteFinish(write.num_rows); + }, + [this](const Status& err) { SetError(err); })); + } + if (hit_backpressure) { + Future<> maybe_backpressure = statistics_.backpressure(); + // It's possible the backpressure was relieved since we last checked + if (maybe_backpressure.is_valid()) { + return maybe_backpressure.Then( + [this, batch, directory] { return DoWriteRecordBatch(batch, directory); }); + } else { + return DoWriteRecordBatch(batch, directory); + } + } + return Future<>::MakeFinished(); + } + + void SetError(Status st) { + std::lock_guard lg(mutex_); + err_ = std::move(st); + } + + Status CheckError() { + std::lock_guard lg(mutex_); + return err_; + } + + FileSystemDatasetWriteOptions write_options_; + DatasetWriterStatistics statistics_; + std::unordered_map> + directory_queues_; + std::mutex mutex_; + // A mutex to guard access to the visitor callbacks + std::mutex visitors_mutex_; + Status err_; +}; + +DatasetWriter::DatasetWriter(FileSystemDatasetWriteOptions write_options, + uint64_t max_rows_queued) + : AsyncCloseablePimpl(), + impl_(new DatasetWriterImpl(std::move(write_options), max_rows_queued)) { + AsyncCloseablePimpl::Init(impl_.get()); +} + +Result>> +DatasetWriter::Make(util::Nursery* nursery, FileSystemDatasetWriteOptions write_options, + uint64_t max_rows_queued) { + RETURN_NOT_OK(ValidateBasenameTemplate(write_options.basename_template)); + RETURN_NOT_OK(EnsureDestinationValid(write_options)); + return nursery->MakeUniqueCloseable(std::move(write_options), + max_rows_queued); +} + +DatasetWriter::~DatasetWriter() = default; + +Future<> DatasetWriter::WriteRecordBatch(std::shared_ptr batch, + const std::string& directory) { + return impl_->WriteRecordBatch(std::move(batch), directory); +} + +} // namespace dataset +} // namespace arrow diff --git a/cpp/src/arrow/dataset/dataset_writer.h b/cpp/src/arrow/dataset/dataset_writer.h new file mode 100644 index 00000000000..427610e853f --- /dev/null +++ b/cpp/src/arrow/dataset/dataset_writer.h @@ -0,0 +1,95 @@ +// 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. + +#pragma once + +#include + +#include "arrow/dataset/file_base.h" +#include "arrow/record_batch.h" +#include "arrow/status.h" +#include "arrow/util/async_nursery.h" +#include "arrow/util/future.h" + +namespace arrow { +namespace dataset { + +constexpr uint64_t kDefaultDatasetWriterMaxRowsQueued = 64 * 1024 * 1024; + +/// \brief Utility class that manages a set of writers to different paths +/// +/// Writers may be closed and reopened (and a new file created) based on the dataset +/// write options (for example, min_rows_per_file or max_open_files) +/// +/// The dataset writer enforces its own back pressure based on the # of rows (as opposed +/// to # of batches which is how it is typically enforced elsewhere) and # of files. +class ARROW_EXPORT DatasetWriter : public util::AsyncCloseablePimpl { + public: + /// \brief Creates a dataset writer + /// + /// Will fail if basename_template is invalid or if there is existing data and + /// existing_data_behavior is kError + /// + /// \param nursery an enclosing nursery which will contain the dataset write + /// \param write_options options to control how the data should be written + /// \param max_rows_queued max # of rows allowed to be queued before the dataset_writer + /// will ask for backpressure + static Result>> + Make(util::Nursery* nursery, FileSystemDatasetWriteOptions write_options, + uint64_t max_rows_queued = kDefaultDatasetWriterMaxRowsQueued); + + ~DatasetWriter(); + + /// \brief Writes a batch to the dataset + /// \param[in] batch The batch to write + /// \param[in] directory The directory to write to + /// + /// Note: The written filename will be {directory}/{filename_factory(i)} where i is a + /// counter controlled by `max_open_files` and `max_rows_per_file` + /// + /// If multiple WriteRecordBatch calls arrive with the same `directory` then the batches + /// may be written to the same file. + /// + /// The returned future will be marked finished when the record batch has been queued + /// to be written. If the returned future is unfinished then this indicates the dataset + /// writer's queue is full and the data provider should pause. + /// + /// This method is NOT async reentrant. The returned future will only be incomplete + /// if back pressure needs to be applied. Async reentrancy is not necessary for + /// concurrent writes to happen. Calling this method again before the previous future + /// completes will not just violate max_rows_queued but likely lead to race conditions. + /// + /// One thing to note is that the ordering of your data can affect your maximum + /// potential parallelism. If this seems odd then consider a dataset where the first + /// 1000 batches go to the same directory and then the 1001st batch goes to a different + /// directory. The only way to get two parallel writes immediately would be to queue + /// all 1000 pending writes to the first directory. + Future<> WriteRecordBatch(std::shared_ptr batch, + const std::string& directory); + + protected: + DatasetWriter(FileSystemDatasetWriteOptions write_options, + uint64_t max_rows_queued = kDefaultDatasetWriterMaxRowsQueued); + + class DatasetWriterImpl; + std::unique_ptr impl_; + + friend util::Nursery; +}; + +} // namespace dataset +} // namespace arrow diff --git a/cpp/src/arrow/dataset/dataset_writer_test.cc b/cpp/src/arrow/dataset/dataset_writer_test.cc new file mode 100644 index 00000000000..0bb78631b59 --- /dev/null +++ b/cpp/src/arrow/dataset/dataset_writer_test.cc @@ -0,0 +1,325 @@ +// 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 "arrow/dataset/dataset_writer.h" + +#include +#include +#include + +#include "arrow/dataset/file_ipc.h" +#include "arrow/filesystem/mockfs.h" +#include "arrow/filesystem/test_util.h" +#include "arrow/testing/future_util.h" +#include "arrow/testing/gtest_util.h" +#include "arrow/util/async_nursery.h" +#include "arrow/util/optional.h" +#include "gtest/gtest.h" + +namespace arrow { +namespace dataset { + +using arrow::fs::internal::MockFileInfo; +using arrow::fs::internal::MockFileSystem; + +struct ExpectedFile { + std::string filename; + uint64_t start; + uint64_t num_rows; +}; + +class DatasetWriterTestFixture : public testing::Test { + protected: + void SetUp() override { + fs::TimePoint mock_now = std::chrono::system_clock::now(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + MockFileSystem::Make(mock_now, {::arrow::fs::Dir("testdir")})); + filesystem_ = std::dynamic_pointer_cast(fs); + schema_ = schema({field("int64", int64())}); + write_options_.filesystem = filesystem_; + write_options_.basename_template = "part-{i}.arrow"; + write_options_.base_dir = "testdir"; + write_options_.writer_pre_finish = [this](FileWriter* writer) { + pre_finish_visited_.push_back(writer->destination().path); + return Status::OK(); + }; + write_options_.writer_post_finish = [this](FileWriter* writer) { + post_finish_visited_.push_back(writer->destination().path); + return Status::OK(); + }; + std::shared_ptr format = std::make_shared(); + write_options_.file_write_options = format->DefaultWriteOptions(); + } + + std::shared_ptr UseGatedFs() { + fs::TimePoint mock_now = std::chrono::system_clock::now(); + auto fs = std::make_shared(mock_now); + ARROW_EXPECT_OK(fs->CreateDir("testdir")); + write_options_.filesystem = fs; + filesystem_ = fs; + return fs; + } + + std::shared_ptr MakeBatch(uint64_t num_rows) { + Int64Builder builder; + for (uint64_t i = counter_; i < counter_ + num_rows; i++) { + ARROW_EXPECT_OK(builder.Append(i)); + } + EXPECT_OK_AND_ASSIGN(std::shared_ptr arr, builder.Finish()); + counter_ += num_rows; + return RecordBatch::Make(schema_, static_cast(num_rows), {std::move(arr)}); + } + + util::optional FindFile(const std::string& filename) { + for (const auto& mock_file : filesystem_->AllFiles()) { + if (mock_file.full_path == filename) { + return mock_file; + } + } + return util::nullopt; + } + + void AssertVisited(const std::vector& actual_paths, + std::string expected_path) { + std::vector::const_iterator found = + std::find(actual_paths.begin(), actual_paths.end(), expected_path); + ASSERT_NE(found, actual_paths.end()) + << "The file " << expected_path << " was not in the list of files visited"; + } + + void AssertFiles(const std::vector& expected_files) { + for (const auto& expected_file : expected_files) { + util::optional written_file = FindFile(expected_file.filename); + ASSERT_TRUE(written_file.has_value()) + << "The file " << expected_file.filename << " was not created"; + { + SCOPED_TRACE("pre_finish"); + AssertVisited(pre_finish_visited_, expected_file.filename); + } + { + SCOPED_TRACE("post_finish"); + AssertVisited(post_finish_visited_, expected_file.filename); + } + // FIXME Check contents + } + } + + void AssertNotFiles(const std::vector expected_non_files) { + for (const auto& expected_non_file : expected_non_files) { + util::optional file = FindFile(expected_non_file); + ASSERT_FALSE(file.has_value()); + } + } + + void AssertEmptyFiles(const std::vector expected_empty_files) { + for (const auto& expected_empty_file : expected_empty_files) { + util::optional file = FindFile(expected_empty_file); + ASSERT_TRUE(file.has_value()); + ASSERT_EQ("", file->data); + } + } + + std::shared_ptr filesystem_; + std::shared_ptr schema_; + std::vector pre_finish_visited_; + std::vector post_finish_visited_; + FileSystemDatasetWriteOptions write_options_; + uint64_t counter_ = 0; +}; + +TEST_F(DatasetWriterTestFixture, Basic) { + ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { + EXPECT_OK_AND_ASSIGN(auto dataset_writer, + DatasetWriter::Make(nursery, write_options_)); + Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), ""); + AssertFinished(queue_fut); + })); + AssertFiles({{"testdir/part-0.arrow", 0, 100}}); +} + +TEST_F(DatasetWriterTestFixture, MaxRows) { + write_options_.max_rows_per_file = 10; + ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { + EXPECT_OK_AND_ASSIGN(auto dataset_writer, + DatasetWriter::Make(nursery, write_options_)); + Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(35), ""); + AssertFinished(queue_fut); + })); + AssertFiles({{"testdir/part-0.arrow", 0, 10}, + {"testdir/part-1.arrow", 10, 20}, + {"testdir/part-2.arrow", 20, 30}, + {"testdir/part-3.arrow", 30, 35}}); + ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { + std::shared_ptr dataset_writer = + nursery->MakeSharedCloseable(write_options_); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); + })); + AssertFiles({{"testdir/part-0.arrow", 0, 10}, {"testdir/part-1.arrow", 10, 8}}); +} + +TEST_F(DatasetWriterTestFixture, ConcurrentWritesSameFile) { + auto gated_fs = UseGatedFs(); + ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { + EXPECT_OK_AND_ASSIGN(auto dataset_writer, + DatasetWriter::Make(nursery, write_options_)); + for (int i = 0; i < 10; i++) { + Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(10), ""); + AssertFinished(queue_fut); + ASSERT_FINISHES_OK(queue_fut); + } + ASSERT_OK(gated_fs->WaitForOpenOutputStream(1)); + ASSERT_OK(gated_fs->UnlockOpenOutputStream(1)); + })); + AssertFiles({{"testdir/part-0.arrow", 0, 100}}); +} + +TEST_F(DatasetWriterTestFixture, ConcurrentWritesDifferentFiles) { + // NBATCHES must be less than I/O executor concurrency to avoid deadlock / test failure + constexpr int NBATCHES = 6; + auto gated_fs = UseGatedFs(); + std::vector expected_files; + ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { + EXPECT_OK_AND_ASSIGN(auto dataset_writer, + DatasetWriter::Make(nursery, write_options_)); + for (int i = 0; i < NBATCHES; i++) { + std::string i_str = std::to_string(i); + expected_files.push_back(ExpectedFile{"testdir/part" + i_str + "/part-0.arrow", + static_cast(i) * 10, + (static_cast(i + 1) * 10)}); + Future<> queue_fut = + dataset_writer->WriteRecordBatch(MakeBatch(10), "part" + i_str); + AssertFinished(queue_fut); + ASSERT_FINISHES_OK(queue_fut); + } + ASSERT_OK(gated_fs->WaitForOpenOutputStream(NBATCHES)); + ASSERT_OK(gated_fs->UnlockOpenOutputStream(NBATCHES)); + })); + AssertFiles(expected_files); +} + +TEST_F(DatasetWriterTestFixture, MaxOpenFiles) { + auto gated_fs = UseGatedFs(); + write_options_.max_open_files = 2; + ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { + EXPECT_OK_AND_ASSIGN(auto dataset_writer, + DatasetWriter::Make(nursery, write_options_)); + + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(10), "part0")); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(10), "part1")); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(10), "part0")); + Future<> fut = dataset_writer->WriteRecordBatch(MakeBatch(10), "part2"); + // Backpressure will be applied until an existing file can be evicted + AssertNotFinished(fut); + + // Ungate the writes to relieve the pressure, testdir/part0 should be closed + ASSERT_OK(gated_fs->WaitForOpenOutputStream(2)); + ASSERT_OK(gated_fs->UnlockOpenOutputStream(4)); + ASSERT_FINISHES_OK(fut); + + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(10), "part0")); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(10), "part1")); + })); + AssertFiles({{"testdir/part0/part-0.arrow", 0, 10}, + {"testdir/part0/part-0.arrow", 20, 10}, + {"testdir/part0/part-1.arrow", 40, 10}, + {"testdir/part1/part-0.arrow", 10, 10}, + {"testdir/part1/part-0.arrow", 50, 10}, + {"testdir/part2/part-0.arrow", 30, 10}}); +} + +TEST_F(DatasetWriterTestFixture, DeleteExistingData) { + fs::TimePoint mock_now = std::chrono::system_clock::now(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + MockFileSystem::Make(mock_now, {::arrow::fs::Dir("testdir"), + fs::File("testdir/part-5.arrow"), + fs::File("testdir/blah.txt")})); + filesystem_ = std::dynamic_pointer_cast(fs); + write_options_.filesystem = filesystem_; + write_options_.existing_data_behavior = kDeleteMatchingPartitions; + ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { + EXPECT_OK_AND_ASSIGN(auto dataset_writer, + DatasetWriter::Make(nursery, write_options_)); + Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), ""); + AssertFinished(queue_fut); + })); + AssertFiles({{"testdir/part-0.arrow", 0, 100}}); + AssertNotFiles({"testdir/part-5.arrow", "testdir/blah.txt"}); +} + +TEST_F(DatasetWriterTestFixture, PartitionedDeleteExistingData) { + fs::TimePoint mock_now = std::chrono::system_clock::now(); + ASSERT_OK_AND_ASSIGN( + std::shared_ptr fs, + MockFileSystem::Make( + mock_now, {::arrow::fs::Dir("testdir"), fs::File("testdir/part0/foo.arrow"), + fs::File("testdir/part1/bar.arrow")})); + filesystem_ = std::dynamic_pointer_cast(fs); + write_options_.filesystem = filesystem_; + write_options_.existing_data_behavior = kDeleteMatchingPartitions; + ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { + EXPECT_OK_AND_ASSIGN(auto dataset_writer, + DatasetWriter::Make(nursery, write_options_)); + Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), "part0"); + AssertFinished(queue_fut); + })); + AssertFiles({{"testdir/part0/part-0.arrow", 0, 100}}); + AssertNotFiles({"testdir/part0/foo.arrow"}); + AssertEmptyFiles({"testdir/part1/bar.arrow"}); +} + +TEST_F(DatasetWriterTestFixture, LeaveExistingData) { + fs::TimePoint mock_now = std::chrono::system_clock::now(); + ASSERT_OK_AND_ASSIGN( + std::shared_ptr fs, + MockFileSystem::Make( + mock_now, {::arrow::fs::Dir("testdir"), fs::File("testdir/part-0.arrow"), + fs::File("testdir/part-5.arrow"), fs::File("testdir/blah.txt")})); + filesystem_ = std::dynamic_pointer_cast(fs); + write_options_.filesystem = filesystem_; + write_options_.existing_data_behavior = kOverwriteOrIgnore; + ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { + EXPECT_OK_AND_ASSIGN(auto dataset_writer, + DatasetWriter::Make(nursery, write_options_)); + Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), ""); + AssertFinished(queue_fut); + })); + AssertFiles({{"testdir/part-0.arrow", 0, 100}}); + AssertEmptyFiles({"testdir/part-5.arrow", "testdir/blah.txt"}); +} + +TEST_F(DatasetWriterTestFixture, ErrOnExistingData) { + fs::TimePoint mock_now = std::chrono::system_clock::now(); + ASSERT_OK_AND_ASSIGN( + std::shared_ptr fs, + MockFileSystem::Make( + mock_now, {::arrow::fs::Dir("testdir"), fs::File("testdir/part-0.arrow"), + fs::File("testdir/part-5.arrow"), fs::File("testdir/blah.txt")})); + filesystem_ = std::dynamic_pointer_cast(fs); + write_options_.filesystem = filesystem_; + ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { + ASSERT_RAISES(Invalid, DatasetWriter::Make(nursery, write_options_)); + })); + AssertEmptyFiles({"testdir/part-0.arrow", "testdir/part-5.arrow", "testdir/blah.txt"}); +} + +} // namespace dataset +} // namespace arrow diff --git a/cpp/src/arrow/dataset/file_base.cc b/cpp/src/arrow/dataset/file_base.cc index 39f0aa7f627..d66f6e7413a 100644 --- a/cpp/src/arrow/dataset/file_base.cc +++ b/cpp/src/arrow/dataset/file_base.cc @@ -18,29 +18,26 @@ #include "arrow/dataset/file_base.h" #include -#include #include -#include #include #include "arrow/compute/exec/forest_internal.h" #include "arrow/compute/exec/subtree_internal.h" #include "arrow/dataset/dataset_internal.h" +#include "arrow/dataset/dataset_writer.h" #include "arrow/dataset/scanner.h" #include "arrow/dataset/scanner_internal.h" #include "arrow/filesystem/filesystem.h" -#include "arrow/filesystem/localfs.h" #include "arrow/filesystem/path_util.h" #include "arrow/io/compressed.h" #include "arrow/io/interfaces.h" #include "arrow/io/memory.h" +#include "arrow/util/async_nursery.h" #include "arrow/util/compression.h" #include "arrow/util/iterator.h" -#include "arrow/util/logging.h" #include "arrow/util/macros.h" #include "arrow/util/make_unique.h" #include "arrow/util/map.h" -#include "arrow/util/mutex.h" #include "arrow/util/string.h" #include "arrow/util/task_group.h" #include "arrow/util/variant.h" @@ -70,7 +67,6 @@ Result> FileSource::OpenCompressed( if (!compression.has_value()) { // Guess compression from file extension auto extension = fs::internal::GetAbstractPathExtension(path()); - util::string_view file_path(path()); if (extension == "gz") { actual_compression = Compression::type::GZIP; } else { @@ -115,7 +111,7 @@ Result> FileFormat::MakeFragment( } // TODO(ARROW-12355[CSV], ARROW-11772[IPC], ARROW-11843[Parquet]) The following -// implementation of ScanBatchesAsync is both ugly and terribly ineffecient. Each of the +// implementation of ScanBatchesAsync is both ugly and terribly inefficient. Each of the // formats should provide their own efficient implementation. Result FileFormat::ScanBatchesAsync( const std::shared_ptr& scan_options, @@ -327,222 +323,73 @@ Status FileWriter::Finish() { namespace { -constexpr util::string_view kIntegerToken = "{i}"; +Future<> WriteNextBatch(DatasetWriter* dataset_writer, TaggedRecordBatch batch, + const FileSystemDatasetWriteOptions& write_options) { + ARROW_ASSIGN_OR_RAISE(auto groups, + write_options.partitioning->Partition(batch.record_batch)); + batch.record_batch.reset(); // drop to hopefully conserve memory -Status ValidateBasenameTemplate(util::string_view basename_template) { - if (basename_template.find(fs::internal::kSep) != util::string_view::npos) { - return Status::Invalid("basename_template contained '/'"); - } - size_t token_start = basename_template.find(kIntegerToken); - if (token_start == util::string_view::npos) { - return Status::Invalid("basename_template did not contain '", kIntegerToken, "'"); - } - return Status::OK(); -} - -/// WriteQueue allows batches to be pushed from multiple threads while another thread -/// flushes some to disk. -class WriteQueue { - public: - WriteQueue(std::string partition_expression, size_t index, - std::shared_ptr schema) - : partition_expression_(std::move(partition_expression)), - index_(index), - schema_(std::move(schema)) {} - - // Push a batch into the writer's queue of pending writes. - void Push(std::shared_ptr batch) { - auto push_lock = push_mutex_.Lock(); - pending_.push_back(std::move(batch)); - } - - // Flush all pending batches, or return immediately if another thread is already - // flushing this queue. - Status Flush(const FileSystemDatasetWriteOptions& write_options) { - if (auto writer_lock = writer_mutex_.TryLock()) { - if (writer_ == nullptr) { - // FileWriters are opened lazily to avoid blocking access to a scan-wide queue set - RETURN_NOT_OK(OpenWriter(write_options)); - } - - while (true) { - std::shared_ptr batch; - { - auto push_lock = push_mutex_.Lock(); - if (pending_.empty()) { - // Ensure the writer_lock is released before the push_lock. Otherwise another - // thread might successfully Push() a batch but then fail to Flush() it since - // the writer_lock is still held, leaving an unflushed batch in pending_. - writer_lock.Unlock(); - break; - } - batch = std::move(pending_.front()); - pending_.pop_front(); - } - RETURN_NOT_OK(writer_->Write(batch)); - } - } - return Status::OK(); - } - - const std::shared_ptr& writer() const { return writer_; } - - private: - Status OpenWriter(const FileSystemDatasetWriteOptions& write_options) { - auto dir = - fs::internal::EnsureTrailingSlash(write_options.base_dir) + partition_expression_; - - auto basename = ::arrow::internal::Replace(write_options.basename_template, - kIntegerToken, std::to_string(index_)); - if (!basename) { - return Status::Invalid("string interpolation of basename template failed"); - } - - auto path = fs::internal::ConcatAbstractPath(dir, *basename); - - RETURN_NOT_OK(write_options.filesystem->CreateDir(dir)); - ARROW_ASSIGN_OR_RAISE(auto destination, - write_options.filesystem->OpenOutputStream(path)); - - ARROW_ASSIGN_OR_RAISE( - writer_, write_options.format()->MakeWriter(std::move(destination), schema_, - write_options.file_write_options, - {write_options.filesystem, path})); - return Status::OK(); - } - - util::Mutex writer_mutex_; - std::shared_ptr writer_; - - util::Mutex push_mutex_; - std::deque> pending_; - - // The (formatted) partition expression to which this queue corresponds - std::string partition_expression_; - - size_t index_; - - std::shared_ptr schema_; -}; - -struct WriteState { - explicit WriteState(FileSystemDatasetWriteOptions write_options) - : write_options(std::move(write_options)) {} - - FileSystemDatasetWriteOptions write_options; - util::Mutex mutex; - std::unordered_map> queues; -}; - -Status WriteNextBatch(WriteState* state, const std::shared_ptr& fragment, - std::shared_ptr batch) { - ARROW_ASSIGN_OR_RAISE(auto groups, state->write_options.partitioning->Partition(batch)); - batch.reset(); // drop to hopefully conserve memory - - if (groups.batches.size() > static_cast(state->write_options.max_partitions)) { + if (groups.batches.size() > static_cast(write_options.max_partitions)) { return Status::Invalid("Fragment would be written into ", groups.batches.size(), " partitions. This exceeds the maximum of ", - state->write_options.max_partitions); + write_options.max_partitions); } - std::unordered_set need_flushed; - for (size_t i = 0; i < groups.batches.size(); ++i) { - auto partition_expression = - and_(std::move(groups.expressions[i]), fragment->partition_expression()); - auto batch = std::move(groups.batches[i]); - - ARROW_ASSIGN_OR_RAISE( - auto part, state->write_options.partitioning->Format(partition_expression)); - - WriteQueue* queue; - { - // lookup the queue to which batch should be appended - auto queues_lock = state->mutex.Lock(); - - queue = ::arrow::internal::GetOrInsertGenerated( - &state->queues, std::move(part), - [&](const std::string& emplaced_part) { - // lookup in `queues` also failed, - // generate a new WriteQueue - size_t queue_index = state->queues.size() - 1; - - return ::arrow::internal::make_unique( - emplaced_part, queue_index, batch->schema()); - }) - ->second.get(); - } - - queue->Push(std::move(batch)); - need_flushed.insert(queue); - } + std::shared_ptr counter = std::make_shared(0); + std::shared_ptr fragment = std::move(batch.fragment); - // flush all touched WriteQueues - for (auto queue : need_flushed) { - RETURN_NOT_OK(queue->Flush(state->write_options)); - } - return Status::OK(); -} - -Status WriteInternal(const ScanOptions& scan_options, WriteState* state, - ScanTaskVector scan_tasks) { - // Store a mapping from partitions (represened by their formatted partition expressions) - // to a WriteQueue which flushes batches into that partition's output file. In principle - // any thread could produce a batch for any partition, so each task alternates between - // pushing batches and flushing them to disk. - auto task_group = scan_options.TaskGroup(); - - for (const auto& scan_task : scan_tasks) { - task_group->Append([&, scan_task] { - std::function)> visitor = - [&](std::shared_ptr batch) { - return WriteNextBatch(state, scan_task->fragment(), std::move(batch)); - }; - return ::arrow::internal::RunSynchronously>( - [&](Executor* executor) { return scan_task->SafeVisit(executor, visitor); }, - /*use_threads=*/false); + AsyncGenerator> partitioned_batch_gen = + [groups, counter, fragment, &write_options, + dataset_writer]() -> Future> { + auto index = *counter; + if (index >= groups.batches.size()) { + return AsyncGeneratorEnd>(); + } + auto partition_expression = + and_(groups.expressions[index], fragment->partition_expression()); + auto next_batch = groups.batches[index]; + ARROW_ASSIGN_OR_RAISE(std::string destination, + write_options.partitioning->Format(partition_expression)); + (*counter)++; + return dataset_writer->WriteRecordBatch(next_batch, destination).Then([next_batch] { + return next_batch; }); - } - return task_group->Finish(); + }; + + return VisitAsyncGenerator( + std::move(partitioned_batch_gen), + [](const std::shared_ptr&) -> Status { return Status::OK(); }); } } // namespace Status FileSystemDataset::Write(const FileSystemDatasetWriteOptions& write_options, std::shared_ptr scanner) { - RETURN_NOT_OK(ValidateBasenameTemplate(write_options.basename_template)); - - // Things we'll un-lazy for the sake of simplicity, with the tradeoff they represent: - // - // - Fragment iteration. Keeping this lazy would allow us to start partitioning/writing - // any fragments we have before waiting for discovery to complete. This isn't - // currently implemented for FileSystemDataset anyway: ARROW-8613 - // - // - ScanTask iteration. Keeping this lazy would save some unnecessary blocking when - // writing Fragments which produce scan tasks slowly. No Fragments do this. - // - // NB: neither of these will have any impact whatsoever on the common case of writing - // an in-memory table to disk. - - ARROW_SUPPRESS_DEPRECATION_WARNING - - // TODO(ARROW-11782/ARROW-12288) Remove calls to Scan() - ARROW_ASSIGN_OR_RAISE(auto scan_task_it, scanner->Scan()); - ARROW_ASSIGN_OR_RAISE(ScanTaskVector scan_tasks, scan_task_it.ToVector()); - - ARROW_UNSUPPRESS_DEPRECATION_WARNING - - WriteState state(write_options); - RETURN_NOT_OK(WriteInternal(*scanner->options(), &state, std::move(scan_tasks))); - - auto task_group = scanner->options()->TaskGroup(); - for (const auto& part_queue : state.queues) { - task_group->Append([&] { - RETURN_NOT_OK(write_options.writer_pre_finish(part_queue.second->writer().get())); - RETURN_NOT_OK(part_queue.second->writer()->Finish()); - return write_options.writer_post_finish(part_queue.second->writer().get()); - }); - } - return task_group->Finish(); + std::function task = [&](util::Nursery* nursery) { + ARROW_ASSIGN_OR_RAISE(auto batch_gen, scanner->ScanBatchesAsync()); + ARROW_ASSIGN_OR_RAISE(auto dataset_writer, + DatasetWriter::Make(nursery, write_options)); + + AsyncGenerator> queued_batch_gen = + [batch_gen, &dataset_writer, &write_options]() -> Future> { + Future next_batch_fut = batch_gen(); + return next_batch_fut.Then( + [&dataset_writer, &write_options](const TaggedRecordBatch& batch) { + if (IsIterationEnd(batch)) { + return AsyncGeneratorEnd>(); + } + return WriteNextBatch(dataset_writer.get(), batch, write_options).Then([] { + return std::make_shared(0); + }); + }); + }; + Future<> queue_fut = + VisitAsyncGenerator(std::move(queued_batch_gen), + [&](const std::shared_ptr&) { return Status::OK(); }); + + return queue_fut.status(); + }; + return util::Nursery::RunInNursery(std::move(task)); } } // namespace dataset diff --git a/cpp/src/arrow/dataset/file_base.h b/cpp/src/arrow/dataset/file_base.h index 49b700a7ef9..ba03dd0ad27 100644 --- a/cpp/src/arrow/dataset/file_base.h +++ b/cpp/src/arrow/dataset/file_base.h @@ -343,6 +343,18 @@ class ARROW_DS_EXPORT FileWriter { fs::FileLocator destination_locator_; }; +/// \brief Controls what happens if files exist in an output directory during a dataset +/// write +enum ExistingDataBehavior : int8_t { + /// Deletes all files in a directory the first time that directory is encountered + kDeleteMatchingPartitions, + /// Ignores existing files, overwriting any that happen to have the same name as an + /// output file + kOverwriteOrIgnore, + /// Returns an error if there are any files or subdirectories in the output directory + kError, +}; + /// \brief Options for writing a dataset. struct ARROW_DS_EXPORT FileSystemDatasetWriteOptions { /// Options for individual fragment writing. @@ -364,6 +376,18 @@ struct ARROW_DS_EXPORT FileSystemDatasetWriteOptions { /// {i} will be replaced by an auto incremented integer. std::string basename_template; + /// If greater than 0 then this will limit the maximum number of files that can be left + /// open. If an attempt is made to open too many files then the least recently used file + /// will be closed. If this setting is set too low you may end up fragmenting your data + /// into many small files. + uint32_t max_open_files = 1024; + + /// If greater than 0 then this will limit how many rows are placed in any single file. + uint64_t max_rows_per_file = 0; + + /// Controls what happens if an output directory already exists. + ExistingDataBehavior existing_data_behavior = kError; + /// Callback to be invoked against all FileWriters before /// they are finalized with FileWriter::Finish(). std::function writer_pre_finish = [](FileWriter*) { diff --git a/cpp/src/arrow/dataset/file_ipc.cc b/cpp/src/arrow/dataset/file_ipc.cc index 17d05b55e01..e01373e79c3 100644 --- a/cpp/src/arrow/dataset/file_ipc.cc +++ b/cpp/src/arrow/dataset/file_ipc.cc @@ -282,9 +282,6 @@ Result> IpcFileFormat::MakeWriter( auto ipc_options = checked_pointer_cast(options); - // override use_threads to avoid nested parallelism - ipc_options->options->use_threads = false; - ARROW_ASSIGN_OR_RAISE(auto writer, ipc::MakeFileWriter(destination, schema, *ipc_options->options, ipc_options->metadata)); diff --git a/cpp/src/arrow/dataset/file_ipc_test.cc b/cpp/src/arrow/dataset/file_ipc_test.cc index 33c3738e72f..cb625a9e1db 100644 --- a/cpp/src/arrow/dataset/file_ipc_test.cc +++ b/cpp/src/arrow/dataset/file_ipc_test.cc @@ -126,7 +126,9 @@ TEST_F(TestIpcFileSystemDataset, WriteExceedsMaxPartitions) { // require that no batch be grouped into more than 2 written batches: write_options_.max_partitions = 2; - EXPECT_OK_AND_ASSIGN(auto scanner, ScannerBuilder(dataset_, scan_options_).Finish()); + auto scanner_builder = ScannerBuilder(dataset_, scan_options_); + ASSERT_OK(scanner_builder.UseAsync(true)); + EXPECT_OK_AND_ASSIGN(auto scanner, scanner_builder.Finish()); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, testing::HasSubstr("This exceeds the maximum"), FileSystemDataset::Write(write_options_, scanner)); } diff --git a/cpp/src/arrow/dataset/file_test.cc b/cpp/src/arrow/dataset/file_test.cc index ab24257cee8..db2eedbcf16 100644 --- a/cpp/src/arrow/dataset/file_test.cc +++ b/cpp/src/arrow/dataset/file_test.cc @@ -319,6 +319,7 @@ TEST_F(TestFileSystemDataset, WriteProjected) { ASSERT_OK(scanner_builder->Project( {compute::call("add", {compute::field_ref("a"), compute::literal(1)})}, {"a_plus_one"})); + ASSERT_OK(scanner_builder->UseAsync(true)); ASSERT_OK_AND_ASSIGN(auto scanner, scanner_builder->Finish()); ASSERT_OK(FileSystemDataset::Write(write_options, scanner)); diff --git a/cpp/src/arrow/dataset/test_util.h b/cpp/src/arrow/dataset/test_util.h index 2d2cfb5de8d..993fbd37144 100644 --- a/cpp/src/arrow/dataset/test_util.h +++ b/cpp/src/arrow/dataset/test_util.h @@ -1086,6 +1086,7 @@ class WriteFileSystemDatasetMixin : public MakeFileSystemDatasetMixin { void DoWrite(std::shared_ptr desired_partitioning) { write_options_.partitioning = desired_partitioning; auto scanner_builder = ScannerBuilder(dataset_, scan_options_); + ASSERT_OK(scanner_builder.UseAsync(true)); ASSERT_OK_AND_ASSIGN(auto scanner, scanner_builder.Finish()); ASSERT_OK(FileSystemDataset::Write(write_options_, scanner)); @@ -1115,7 +1116,7 @@ class WriteFileSystemDatasetMixin : public MakeFileSystemDatasetMixin { {"region": "QC", "model": "X", "sales": 1.0, "country": "CA"}, {"region": "QC", "model": "Y", "sales": 69, "country": "CA"} ])"; - expected_files_["/new_root/2019/1/dat_1"] = R"([ + expected_files_["/new_root/2019/1/dat_0"] = R"([ {"region": "CA", "model": "3", "sales": 273.5, "country": "US"}, {"region": "CA", "model": "S", "sales": 13, "country": "US"}, {"region": "CA", "model": "X", "sales": 54, "country": "US"}, @@ -1143,7 +1144,7 @@ class WriteFileSystemDatasetMixin : public MakeFileSystemDatasetMixin { {"year": 2018, "month": 1, "model": "Y", "sales": 27.5}, {"year": 2018, "month": 1, "model": "X", "sales": 136.25} ])"; - expected_files_["/new_root/CA/QC/dat_1"] = R"([ + expected_files_["/new_root/CA/QC/dat_0"] = R"([ {"year": 2018, "month": 1, "model": "3", "sales": 512}, {"year": 2018, "month": 1, "model": "S", "sales": 978}, {"year": 2018, "month": 1, "model": "X", "sales": 1.0}, @@ -1153,7 +1154,7 @@ class WriteFileSystemDatasetMixin : public MakeFileSystemDatasetMixin { {"year": 2019, "month": 1, "model": "X", "sales": 42}, {"year": 2019, "month": 1, "model": "Y", "sales": 37} ])"; - expected_files_["/new_root/US/CA/dat_2"] = R"([ + expected_files_["/new_root/US/CA/dat_0"] = R"([ {"year": 2019, "month": 1, "model": "3", "sales": 273.5}, {"year": 2019, "month": 1, "model": "S", "sales": 13}, {"year": 2019, "month": 1, "model": "X", "sales": 54}, @@ -1177,19 +1178,19 @@ class WriteFileSystemDatasetMixin : public MakeFileSystemDatasetMixin { {"model": "Y", "sales": 27.5}, {"model": "X", "sales": 136.25} ])"; - expected_files_["/new_root/2018/1/CA/QC/dat_1"] = R"([ + expected_files_["/new_root/2018/1/CA/QC/dat_0"] = R"([ {"model": "3", "sales": 512}, {"model": "S", "sales": 978}, {"model": "X", "sales": 1.0}, {"model": "Y", "sales": 69} ])"; - expected_files_["/new_root/2019/1/US/CA/dat_2"] = R"([ + expected_files_["/new_root/2019/1/US/CA/dat_0"] = R"([ {"model": "3", "sales": 273.5}, {"model": "S", "sales": 13}, {"model": "X", "sales": 54}, {"model": "Y", "sales": 21} ])"; - expected_files_["/new_root/2019/1/CA/QC/dat_3"] = R"([ + expected_files_["/new_root/2019/1/CA/QC/dat_0"] = R"([ {"model": "S", "sales": 10}, {"model": "3", "sales": 152.25}, {"model": "X", "sales": 42}, diff --git a/cpp/src/arrow/filesystem/filesystem.cc b/cpp/src/arrow/filesystem/filesystem.cc index fbe8b1f1700..a922f0210a3 100644 --- a/cpp/src/arrow/filesystem/filesystem.cc +++ b/cpp/src/arrow/filesystem/filesystem.cc @@ -15,12 +15,12 @@ // specific language governing permissions and limitations // under the License. +#include "arrow/filesystem/filesystem.h" + #include #include #include "arrow/util/config.h" - -#include "arrow/filesystem/filesystem.h" #ifdef ARROW_HDFS #include "arrow/filesystem/hdfs.h" #endif @@ -176,6 +176,15 @@ Status FileSystem::DeleteFiles(const std::vector& paths) { return st; } +Status FileSystem::DeleteFiles(const FileSelector& select) { + Status st = Status::OK(); + ARROW_ASSIGN_OR_RAISE(auto file_infos, GetFileInfo(select)); + for (const auto& file_info : file_infos) { + st &= DeleteFile(file_info.path()); + } + return st; +} + namespace { Status ValidateInputFileInfo(const FileInfo& info) { diff --git a/cpp/src/arrow/filesystem/filesystem.h b/cpp/src/arrow/filesystem/filesystem.h index 6a36d51e929..0326ed8f95e 100644 --- a/cpp/src/arrow/filesystem/filesystem.h +++ b/cpp/src/arrow/filesystem/filesystem.h @@ -202,7 +202,7 @@ class ARROW_EXPORT FileSystem : public std::enable_shared_from_this /// the returned future to complete before calling the generator again. virtual FileInfoGenerator GetFileInfoGenerator(const FileSelector& select); - /// Create a directory and subdirectories. + /// Create a directory and parent directories. /// /// This function succeeds if the directory already exists. virtual Status CreateDir(const std::string& path, bool recursive = true) = 0; @@ -230,6 +230,9 @@ class ARROW_EXPORT FileSystem : public std::enable_shared_from_this /// The default implementation issues individual delete operations in sequence. virtual Status DeleteFiles(const std::vector& paths); + /// Selector based override of DeleteFiles + virtual Status DeleteFiles(const FileSelector& selector); + /// Move / rename a file or directory. /// /// If the destination exists: diff --git a/cpp/src/arrow/filesystem/hdfs_test.cc b/cpp/src/arrow/filesystem/hdfs_test.cc index 498549b85f0..bd670c4b9ac 100644 --- a/cpp/src/arrow/filesystem/hdfs_test.cc +++ b/cpp/src/arrow/filesystem/hdfs_test.cc @@ -15,14 +15,15 @@ // specific language governing permissions and limitations // under the License. +#include "arrow/filesystem/hdfs.h" + +#include + #include #include #include #include -#include - -#include "arrow/filesystem/hdfs.h" #include "arrow/filesystem/test_util.h" #include "arrow/testing/gtest_util.h" #include "arrow/util/io_util.h" @@ -37,7 +38,7 @@ namespace fs { TEST(TestHdfsOptions, FromUri) { HdfsOptions options; - internal::Uri uri; + Uri uri; ASSERT_OK(uri.Parse("hdfs://localhost")); ASSERT_OK_AND_ASSIGN(options, HdfsOptions::FromUri(uri)); diff --git a/cpp/src/arrow/filesystem/test_util.cc b/cpp/src/arrow/filesystem/test_util.cc index 4fe073c0aa0..af7b5ebb7b0 100644 --- a/cpp/src/arrow/filesystem/test_util.cc +++ b/cpp/src/arrow/filesystem/test_util.cc @@ -15,24 +15,27 @@ // specific language governing permissions and limitations // under the License. +#include +#include + +// Can't import test_util.h before gtest.h because it depends on ASSERT_GE but clang +// wants the "related import" to come first so we need to disable here +// clang-format off +#include "arrow/filesystem/test_util.h" +// clang-format on + #include #include -#include #include #include #include -#include -#include - -#include "arrow/filesystem/test_util.h" -#include "arrow/filesystem/util_internal.h" +#include "arrow/filesystem/mockfs.h" #include "arrow/io/interfaces.h" #include "arrow/status.h" #include "arrow/testing/future_util.h" #include "arrow/testing/gtest_util.h" #include "arrow/util/async_generator.h" -#include "arrow/util/future.h" #include "arrow/util/key_value_metadata.h" #include "arrow/util/vector.h" @@ -167,6 +170,27 @@ void AssertFileInfo(FileSystem* fs, const std::string& path, FileType type, AssertFileInfo(info, path, type, size); } +GatedMockFilesystem::GatedMockFilesystem(TimePoint current_time, + const io::IOContext& io_context) + : internal::MockFileSystem(current_time, io_context) {} +GatedMockFilesystem::~GatedMockFilesystem() = default; + +Result> GatedMockFilesystem::OpenOutputStream( + const std::string& path, const std::shared_ptr& metadata) { + RETURN_NOT_OK(open_output_sem_.Acquire(1)); + return MockFileSystem::OpenOutputStream(path, metadata); +} + +// Wait until at least num_waiters are waiting on OpenOutputStream +Status GatedMockFilesystem::WaitForOpenOutputStream(uint32_t num_waiters) { + return open_output_sem_.WaitForWaiters(num_waiters); +} + +// Unlocks `num_waiters` individual calls to OpenOutputStream +Status GatedMockFilesystem::UnlockOpenOutputStream(uint32_t num_waiters) { + return open_output_sem_.Release(num_waiters); +} + //////////////////////////////////////////////////////////////////////////// // GenericFileSystemTest implementation diff --git a/cpp/src/arrow/filesystem/test_util.h b/cpp/src/arrow/filesystem/test_util.h index 917a768084c..59852bbe39b 100644 --- a/cpp/src/arrow/filesystem/test_util.h +++ b/cpp/src/arrow/filesystem/test_util.h @@ -23,7 +23,9 @@ #include #include "arrow/filesystem/filesystem.h" +#include "arrow/filesystem/mockfs.h" #include "arrow/testing/visibility.h" +#include "arrow/util/counting_semaphore.h" namespace arrow { namespace fs { @@ -38,6 +40,23 @@ static inline FileInfo Dir(std::string path) { return FileInfo(std::move(path), FileType::Directory); } +class ARROW_TESTING_EXPORT GatedMockFilesystem : public internal::MockFileSystem { + public: + GatedMockFilesystem(TimePoint current_time, + const io::IOContext& = io::default_io_context()); + ~GatedMockFilesystem() override; + + Result> OpenOutputStream( + const std::string& path, + const std::shared_ptr& metadata = {}) override; + + Status WaitForOpenOutputStream(uint32_t num_waiters); + Status UnlockOpenOutputStream(uint32_t num_waiters); + + private: + util::CountingSemaphore open_output_sem_; +}; + ARROW_TESTING_EXPORT void CreateFile(FileSystem* fs, const std::string& path, const std::string& data); diff --git a/cpp/src/arrow/testing/gtest_util.cc b/cpp/src/arrow/testing/gtest_util.cc index 24f5edcc6cb..f471ed140a0 100644 --- a/cpp/src/arrow/testing/gtest_util.cc +++ b/cpp/src/arrow/testing/gtest_util.cc @@ -16,6 +16,7 @@ // under the License. #include "arrow/testing/gtest_util.h" + #include "arrow/testing/extension_type.h" #ifndef _WIN32 @@ -908,7 +909,9 @@ ExtensionTypeGuard::~ExtensionTypeGuard() { class GatingTask::Impl : public std::enable_shared_from_this { public: explicit Impl(double timeout_seconds) - : timeout_seconds_(timeout_seconds), status_(), unlocked_(false) {} + : timeout_seconds_(timeout_seconds), status_(), unlocked_(false) { + unlocked_future_ = Future<>::Make(); + } ~Impl() { if (num_running_ != num_launched_) { @@ -930,6 +933,15 @@ class GatingTask::Impl : public std::enable_shared_from_this { return [self] { self->RunTask(); }; } + Future<> AsyncTask() { + num_launched_++; + num_running_++; + /// TODO(ARROW-13004) Could maybe implement this check with future chains + /// if we check to see if the future has been "consumed" or not + num_finished_++; + return unlocked_future_; + } + void RunTask() { std::unique_lock lk(mx_); num_running_++; @@ -942,7 +954,6 @@ class GatingTask::Impl : public std::enable_shared_from_this { " seconds) waiting for the gating task to be unlocked"); } num_finished_++; - finished_cv_.notify_all(); } Status WaitForRunning(int count) { @@ -959,6 +970,7 @@ class GatingTask::Impl : public std::enable_shared_from_this { std::lock_guard lk(mx_); unlocked_ = true; unlocked_cv_.notify_all(); + unlocked_future_.MarkFinished(); return status_; } @@ -972,7 +984,7 @@ class GatingTask::Impl : public std::enable_shared_from_this { std::mutex mx_; std::condition_variable running_cv_; std::condition_variable unlocked_cv_; - std::condition_variable finished_cv_; + Future<> unlocked_future_; }; GatingTask::GatingTask(double timeout_seconds) : impl_(new Impl(timeout_seconds)) {} @@ -981,6 +993,8 @@ GatingTask::~GatingTask() {} std::function GatingTask::Task() { return impl_->Task(); } +Future<> GatingTask::AsyncTask() { return impl_->AsyncTask(); } + Status GatingTask::Unlock() { return impl_->Unlock(); } Status GatingTask::WaitForRunning(int count) { return impl_->WaitForRunning(count); } diff --git a/cpp/src/arrow/testing/gtest_util.h b/cpp/src/arrow/testing/gtest_util.h index 65ab33c5d1f..d04efbf37da 100644 --- a/cpp/src/arrow/testing/gtest_util.h +++ b/cpp/src/arrow/testing/gtest_util.h @@ -17,6 +17,8 @@ #pragma once +#include + #include #include #include @@ -28,8 +30,6 @@ #include #include -#include - #include "arrow/array/builder_binary.h" #include "arrow/array/builder_primitive.h" #include "arrow/array/builder_time.h" @@ -94,7 +94,7 @@ #define ARROW_EXPECT_OK(expr) \ do { \ - auto _res = (expr); \ + const auto& _res = (expr); \ ::arrow::Status _st = ::arrow::internal::GenericToStatus(_res); \ EXPECT_TRUE(_st.ok()) << "'" ARROW_STRINGIFY(expr) "' failed with " \ << _st.ToString(); \ @@ -643,6 +643,9 @@ class ARROW_TESTING_EXPORT GatingTask { /// /// Note: The GatingTask must outlive any Task instances std::function Task(); + /// \brief Creates a new waiting task as a future. The future will not complete + /// until unlocked. + Future<> AsyncTask(); /// \brief Waits until at least count tasks are running. Status WaitForRunning(int count); /// \brief Unlocks all waiting tasks. Returns an invalid status if any waiting task has diff --git a/cpp/src/arrow/util/CMakeLists.txt b/cpp/src/arrow/util/CMakeLists.txt index 1b14215ddd8..bc44ef729d6 100644 --- a/cpp/src/arrow/util/CMakeLists.txt +++ b/cpp/src/arrow/util/CMakeLists.txt @@ -48,6 +48,7 @@ add_arrow_test(utility-test cache_test.cc checked_cast_test.cc compression_test.cc + counting_semaphore_test.cc decimal_test.cc formatting_util_test.cc key_value_metadata_test.cc diff --git a/cpp/src/arrow/util/counting_semaphore.cc b/cpp/src/arrow/util/counting_semaphore.cc new file mode 100644 index 00000000000..b3106a6f824 --- /dev/null +++ b/cpp/src/arrow/util/counting_semaphore.cc @@ -0,0 +1,126 @@ +// 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 "arrow/util/counting_semaphore.h" + +#include +#include +#include +#include +#include + +#include "arrow/status.h" + +namespace arrow { +namespace util { + +class CountingSemaphore::Impl { + public: + Impl(uint32_t initial_avail, double timeout_seconds) + : num_permits_(initial_avail), timeout_seconds_(timeout_seconds) {} + + Status Acquire(uint32_t num_permits) { + std::unique_lock lk(mutex_); + RETURN_NOT_OK(CheckClosed()); + num_waiters_ += num_permits; + waiter_cv_.notify_all(); + bool timed_out = !acquirer_cv_.wait_for( + lk, std::chrono::nanoseconds(static_cast(timeout_seconds_ * 1e9)), + [&] { return closed_ || num_permits <= num_permits_; }); + num_waiters_ -= num_permits; + if (timed_out) { + return Status::Invalid("Timed out waiting for semaphore to release ", num_permits, + " permits."); + } + if (closed_) { + return Status::Invalid("Semaphore closed while acquiring"); + } + num_permits_ -= num_permits; + return Status::OK(); + } + + Status Release(uint32_t num_permits) { + std::lock_guard lg(mutex_); + RETURN_NOT_OK(CheckClosed()); + num_permits_ += num_permits; + acquirer_cv_.notify_all(); + return Status::OK(); + } + + Status WaitForWaiters(uint32_t num_waiters) { + std::unique_lock lk(mutex_); + RETURN_NOT_OK(CheckClosed()); + if (waiter_cv_.wait_for( + lk, std::chrono::nanoseconds(static_cast(timeout_seconds_ * 1e9)), + [&] { return closed_ || num_waiters <= num_waiters_; })) { + if (closed_) { + return Status::Invalid("Semaphore closed while waiting for waiters"); + } + return Status::OK(); + } + return Status::Invalid("Timed out waiting for ", num_waiters, + " to start waiting on semaphore"); + } + + Status Close() { + std::lock_guard lg(mutex_); + RETURN_NOT_OK(CheckClosed()); + closed_ = true; + if (num_waiters_ > 0) { + waiter_cv_.notify_all(); + acquirer_cv_.notify_all(); + return Status::Invalid( + "There were one or more threads waiting on a semaphore when it was closed"); + } + return Status::OK(); + } + + private: + Status CheckClosed() const { + if (closed_) { + return Status::Invalid("Invalid operation on closed semaphore"); + } + return Status::OK(); + } + + uint32_t num_permits_; + double timeout_seconds_; + uint32_t num_waiters_ = 0; + bool closed_ = false; + std::mutex mutex_; + std::condition_variable acquirer_cv_; + std::condition_variable waiter_cv_; +}; + +CountingSemaphore::CountingSemaphore(uint32_t initial_avail, double timeout_seconds) + : impl_(new Impl(initial_avail, timeout_seconds)) {} + +CountingSemaphore::~CountingSemaphore() = default; + +Status CountingSemaphore::Acquire(uint32_t num_permits) { + return impl_->Acquire(num_permits); +} +Status CountingSemaphore::Release(uint32_t num_permits) { + return impl_->Release(num_permits); +} +Status CountingSemaphore::WaitForWaiters(uint32_t num_waiters) { + return impl_->WaitForWaiters(num_waiters); +} +Status CountingSemaphore::Close() { return impl_->Close(); } + +} // namespace util +} // namespace arrow diff --git a/cpp/src/arrow/util/counting_semaphore.h b/cpp/src/arrow/util/counting_semaphore.h new file mode 100644 index 00000000000..317636190ce --- /dev/null +++ b/cpp/src/arrow/util/counting_semaphore.h @@ -0,0 +1,60 @@ +// 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. + +#ifndef ARROW_COUNTING_SEMAPHORE_H +#define ARROW_COUNTING_SEMAPHORE_H + +#include + +#include "arrow/status.h" + +namespace arrow { +namespace util { + +/// \brief Simple mutex-based counting semaphore with timeout +class ARROW_EXPORT CountingSemaphore { + public: + /// \brief Create an instance with initial_avail starting permits + /// + /// \param[in] initial_avail The semaphore will start with this many permits available + /// \param[in] timeout_seconds A timeout to be applied to all operations. Operations + /// will return Status::Invalid if this timeout elapses + explicit CountingSemaphore(uint32_t initial_avail = 0, double timeout_seconds = 10); + ~CountingSemaphore(); + /// \brief Block until num_permits permits are available + Status Acquire(uint32_t num_permits); + /// \brief Make num_permits permits available + Status Release(uint32_t num_permits); + /// \brief Waits until num_waiters are waiting on permits + /// + /// This method is non-standard but useful in unit tests to ensure sequencing + Status WaitForWaiters(uint32_t num_waiters); + /// \brief Immediately times out any waiters + /// + /// This method will return Status::OK only if there were no waiters to time out. + /// Once closed any operation on this instance will return an invalid status. + Status Close(); + + private: + class Impl; + std::unique_ptr impl_; +}; + +} // namespace util +} // namespace arrow + +#endif // ARROW_COUNTING_SEMAPHORE_H diff --git a/cpp/src/arrow/util/counting_semaphore_test.cc b/cpp/src/arrow/util/counting_semaphore_test.cc new file mode 100644 index 00000000000..e5369525fc9 --- /dev/null +++ b/cpp/src/arrow/util/counting_semaphore_test.cc @@ -0,0 +1,98 @@ +// 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 "arrow/util/counting_semaphore.h" + +#include +#include +#include + +#include "arrow/testing/gtest_util.h" +#include "gtest/gtest.h" + +namespace arrow { +namespace util { + +TEST(CountingSemaphore, Basic) { + CountingSemaphore semaphore; + std::atomic acquired{false}; + std::atomic started{false}; + std::thread acquirer([&] { + started.store(true); + ASSERT_OK(semaphore.Acquire(3)); + acquired = true; + }); + ASSERT_OK(semaphore.WaitForWaiters(1)); + ASSERT_TRUE(started.load()); + ASSERT_FALSE(acquired.load()); + ASSERT_OK(semaphore.Release(2)); + SleepABit(); + ASSERT_FALSE(acquired.load()); + ASSERT_OK(semaphore.Release(1)); + BusyWait(10, [&] { return acquired.load(); }); + ASSERT_TRUE(acquired.load()); + ASSERT_OK(semaphore.Close()); + acquirer.join(); +} + +TEST(CountingSemaphore, CloseAborts) { + CountingSemaphore semaphore; + std::atomic cleanup{false}; + std::thread acquirer([&] { + ASSERT_RAISES(Invalid, semaphore.Acquire(1)); + cleanup = true; + }); + ASSERT_OK(semaphore.WaitForWaiters(1)); + ASSERT_FALSE(cleanup.load()); + ASSERT_RAISES(Invalid, semaphore.Close()); + BusyWait(10, [&] { return cleanup.load(); }); + acquirer.join(); +} + +TEST(CountingSemaphore, Stress) { + constexpr uint32_t NTHREADS = 100; + CountingSemaphore semaphore; + std::vector max_allowed_cases = {1, 3}; + std::atomic count{0}; + std::atomic max_exceeded{false}; + std::vector threads; + for (uint32_t max_allowed : max_allowed_cases) { + ASSERT_OK(semaphore.Release(max_allowed)); + for (uint32_t i = 0; i < NTHREADS; i++) { + threads.emplace_back([&] { + ASSERT_OK(semaphore.Acquire(1)); + uint32_t last_count = count.fetch_add(1); + if (last_count >= max_allowed) { + std::cout << last_count << std::endl; + max_exceeded.store(true); + } + SleepABit(); + count.fetch_sub(1); + ASSERT_OK(semaphore.Release(1)); + }); + } + ASSERT_OK(semaphore.Acquire(max_allowed)); + } + for (auto& thread : threads) { + thread.join(); + } + ASSERT_OK(semaphore.Close()); + ASSERT_FALSE(max_exceeded.load()); +} + +} // namespace util +} // namespace arrow diff --git a/cpp/src/jni/dataset/jni_wrapper.cc b/cpp/src/jni/dataset/jni_wrapper.cc index d61fb3f964e..464cd78c709 100644 --- a/cpp/src/jni/dataset/jni_wrapper.cc +++ b/cpp/src/jni/dataset/jni_wrapper.cc @@ -145,7 +145,7 @@ class DisposableScannerAdaptor { static arrow::Result> Create( std::shared_ptr scanner) { - ARROW_ASSIGN_OR_RAISE(auto batch_itr, scanner->ScanBatches()) + ARROW_ASSIGN_OR_RAISE(auto batch_itr, scanner->ScanBatches()); return std::make_shared(scanner, std::move(batch_itr)); } From dac4e0d5ff59fd1038e38c972aebad326866cdd8 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Fri, 3 Sep 2021 16:33:34 -1000 Subject: [PATCH 02/26] ARROW-13542: Adapted to use async smart pointers and async task group instead of the nursery concept --- cpp/src/arrow/dataset/dataset_writer.cc | 92 ++++++----- cpp/src/arrow/dataset/dataset_writer.h | 18 +- cpp/src/arrow/dataset/dataset_writer_test.cc | 164 +++++++++---------- cpp/src/arrow/dataset/file_base.cc | 44 +++-- 4 files changed, 151 insertions(+), 167 deletions(-) diff --git a/cpp/src/arrow/dataset/dataset_writer.cc b/cpp/src/arrow/dataset/dataset_writer.cc index 0b47017a3e4..5feb55ba6d4 100644 --- a/cpp/src/arrow/dataset/dataset_writer.cc +++ b/cpp/src/arrow/dataset/dataset_writer.cc @@ -130,7 +130,7 @@ class DatasetWriterStatistics { std::mutex mutex_; }; -class DatasetWriterFileQueue : public util::AsyncCloseable { +class DatasetWriterFileQueue : public util::AsyncDestroyable { public: explicit DatasetWriterFileQueue(const Future>& writer_fut, const FileSystemDatasetWriteOptions& options, @@ -160,7 +160,7 @@ class DatasetWriterFileQueue : public util::AsyncCloseable { return write_future; } - Future<> DoClose() override { + Future<> DoDestroy() override { std::lock_guard lg(mutex); if (!running_task_.is_valid()) { RETURN_NOT_OK(DoFinish()); @@ -259,15 +259,13 @@ struct WriteTask { uint64_t num_rows; }; -class DatasetWriterDirectoryQueue : public util::AsyncCloseable { +class DatasetWriterDirectoryQueue : public util::AsyncDestroyable { public: - DatasetWriterDirectoryQueue(util::AsyncCloseable* parent, std::string directory, - std::shared_ptr schema, + DatasetWriterDirectoryQueue(std::string directory, std::shared_ptr schema, const FileSystemDatasetWriteOptions& write_options, DatasetWriterStatistics* write_statistics, std::mutex* visitors_mutex) - : util::AsyncCloseable(parent), - directory_(std::move(directory)), + : directory_(std::move(directory)), schema_(std::move(schema)), write_options_(write_options), write_statistics_(write_statistics), @@ -275,8 +273,7 @@ class DatasetWriterDirectoryQueue : public util::AsyncCloseable { Result> NextWritableChunk( std::shared_ptr batch, std::shared_ptr* remainder, - bool* will_open_file) { - RETURN_NOT_OK(CheckClosed()); + bool* will_open_file) const { DCHECK_GT(batch->num_rows(), 0); uint64_t rows_available = std::numeric_limits::max(); *will_open_file = rows_written_ == 0; @@ -295,11 +292,10 @@ class DatasetWriterDirectoryQueue : public util::AsyncCloseable { } Future StartWrite(const std::shared_ptr& batch) { - RETURN_NOT_OK(CheckClosed()); rows_written_ += batch->num_rows(); WriteTask task{current_filename_, static_cast(batch->num_rows())}; if (!latest_open_file_) { - latest_open_file_ = OpenFileQueue(current_filename_); + ARROW_ASSIGN_OR_RAISE(latest_open_file_, OpenFileQueue(current_filename_)); } return latest_open_file_->Push(batch).Then([task] { return task; }); } @@ -330,7 +326,8 @@ class DatasetWriterDirectoryQueue : public util::AsyncCloseable { {write_options_.filesystem, filename}); } - std::shared_ptr OpenFileQueue(const std::string& filename) { + Result> OpenFileQueue( + const std::string& filename) { write_statistics_->RecordFileStart(); Future> file_writer_fut = init_future_.Then([this, filename] { @@ -339,10 +336,10 @@ class DatasetWriterDirectoryQueue : public util::AsyncCloseable { return DeferNotOk( io_executor->Submit([this, filename]() { return OpenWriter(filename); })); }); - auto file_queue = nursery_->MakeSharedCloseable( + auto file_queue = util::MakeSharedAsync( file_writer_fut, write_options_, visitors_mutex_); - AddDependentTask( - file_queue->OnClosed().Then([this] { write_statistics_->RecordFileFinished(); })); + RETURN_NOT_OK(task_group_.AddTask(file_queue->on_closed().Then( + [this] { write_statistics_->RecordFileFinished(); }))); return file_queue; } @@ -363,22 +360,28 @@ class DatasetWriterDirectoryQueue : public util::AsyncCloseable { })); } - Future<> DoClose() override { return FinishCurrentFile(); } - - static Result> Make( - util::AsyncCloseable* parent, util::Nursery* nursery, - const FileSystemDatasetWriteOptions& write_options, - DatasetWriterStatistics* write_statistics, std::shared_ptr schema, - std::string dir, std::mutex* visitors_mutex) { - auto dir_queue = nursery->MakeSharedCloseable( - parent, std::move(dir), std::move(schema), write_options, write_statistics, + static Result>> + Make(util::AsyncTaskGroup* task_group, + const FileSystemDatasetWriteOptions& write_options, + DatasetWriterStatistics* write_statistics, std::shared_ptr schema, + std::string dir, std::mutex* visitors_mutex) { + auto dir_queue = util::MakeUniqueAsync( + std::move(dir), std::move(schema), write_options, write_statistics, visitors_mutex); + RETURN_NOT_OK(task_group->AddTask(dir_queue->on_closed())); dir_queue->PrepareDirectory(); ARROW_ASSIGN_OR_RAISE(dir_queue->current_filename_, dir_queue->GetNextFilename()); return dir_queue; } + Future<> DoDestroy() override { + latest_open_file_.reset(); + return task_group_.WaitForTasksToFinish(); + } + private: + util::AsyncTaskGroup task_group_; std::string directory_; std::shared_ptr schema_; const FileSystemDatasetWriteOptions& write_options_; @@ -424,7 +427,7 @@ Status EnsureDestinationValid(const FileSystemDatasetWriteOptions& options) { } // namespace -class DatasetWriter::DatasetWriterImpl : public util::AsyncCloseable { +class DatasetWriter::DatasetWriterImpl : public util::AsyncDestroyable { public: DatasetWriterImpl(FileSystemDatasetWriteOptions write_options, uint64_t max_rows_queued) : write_options_(std::move(write_options)), @@ -432,7 +435,6 @@ class DatasetWriter::DatasetWriterImpl : public util::AsyncCloseable { Future<> WriteRecordBatch(std::shared_ptr batch, const std::string& directory) { - RETURN_NOT_OK(CheckClosed()); RETURN_NOT_OK(CheckError()); if (!directory.empty()) { auto full_path = @@ -443,11 +445,6 @@ class DatasetWriter::DatasetWriterImpl : public util::AsyncCloseable { } } - Future<> DoClose() override { - directory_queues_.clear(); - return CheckError(); - } - protected: Status CloseLargestFile() { std::shared_ptr largest = nullptr; @@ -468,7 +465,7 @@ class DatasetWriter::DatasetWriterImpl : public util::AsyncCloseable { auto dir_queue_itr, ::arrow::internal::GetOrInsertGenerated( &directory_queues_, directory, [this, &batch](const std::string& dir) { - return DatasetWriterDirectoryQueue::Make(this, nursery_, write_options_, + return DatasetWriterDirectoryQueue::Make(&task_group_, write_options_, &statistics_, batch->schema(), dir, &visitors_mutex_); })); @@ -503,11 +500,11 @@ class DatasetWriter::DatasetWriterImpl : public util::AsyncCloseable { // One of the below callbacks could run immediately and set err_ so we check // it each time through the loop RETURN_NOT_OK(CheckError()); - AddDependentTask(scheduled_write.Then( + RETURN_NOT_OK(task_group_.AddTask(scheduled_write.Then( [this](const WriteTask& write) { statistics_.RecordWriteFinish(write.num_rows); }, - [this](const Status& err) { SetError(err); })); + [this](const Status& err) { SetError(err); }))); } if (hit_backpressure) { Future<> maybe_backpressure = statistics_.backpressure(); @@ -532,6 +529,12 @@ class DatasetWriter::DatasetWriterImpl : public util::AsyncCloseable { return err_; } + Future<> DoDestroy() override { + directory_queues_.clear(); + return task_group_.WaitForTasksToFinish().Then([this] { return err_; }); + } + + util::AsyncTaskGroup task_group_; FileSystemDatasetWriteOptions write_options_; DatasetWriterStatistics statistics_; std::unordered_map> @@ -544,18 +547,15 @@ class DatasetWriter::DatasetWriterImpl : public util::AsyncCloseable { DatasetWriter::DatasetWriter(FileSystemDatasetWriteOptions write_options, uint64_t max_rows_queued) - : AsyncCloseablePimpl(), - impl_(new DatasetWriterImpl(std::move(write_options), max_rows_queued)) { - AsyncCloseablePimpl::Init(impl_.get()); -} + : impl_(util::MakeUniqueAsync(std::move(write_options), + max_rows_queued)) {} -Result>> -DatasetWriter::Make(util::Nursery* nursery, FileSystemDatasetWriteOptions write_options, - uint64_t max_rows_queued) { +Result> DatasetWriter::Make( + FileSystemDatasetWriteOptions write_options, uint64_t max_rows_queued) { RETURN_NOT_OK(ValidateBasenameTemplate(write_options.basename_template)); RETURN_NOT_OK(EnsureDestinationValid(write_options)); - return nursery->MakeUniqueCloseable(std::move(write_options), - max_rows_queued); + return std::unique_ptr( + new DatasetWriter(std::move(write_options), max_rows_queued)); } DatasetWriter::~DatasetWriter() = default; @@ -565,5 +565,11 @@ Future<> DatasetWriter::WriteRecordBatch(std::shared_ptr batch, return impl_->WriteRecordBatch(std::move(batch), directory); } +Future<> DatasetWriter::Finish() { + Future<> finished = impl_->on_closed(); + impl_.reset(); + return finished; +} + } // namespace dataset } // namespace arrow diff --git a/cpp/src/arrow/dataset/dataset_writer.h b/cpp/src/arrow/dataset/dataset_writer.h index 427610e853f..3b9e1c88b35 100644 --- a/cpp/src/arrow/dataset/dataset_writer.h +++ b/cpp/src/arrow/dataset/dataset_writer.h @@ -22,7 +22,7 @@ #include "arrow/dataset/file_base.h" #include "arrow/record_batch.h" #include "arrow/status.h" -#include "arrow/util/async_nursery.h" +#include "arrow/util/async_util.h" #include "arrow/util/future.h" namespace arrow { @@ -37,20 +37,19 @@ constexpr uint64_t kDefaultDatasetWriterMaxRowsQueued = 64 * 1024 * 1024; /// /// The dataset writer enforces its own back pressure based on the # of rows (as opposed /// to # of batches which is how it is typically enforced elsewhere) and # of files. -class ARROW_EXPORT DatasetWriter : public util::AsyncCloseablePimpl { +class ARROW_EXPORT DatasetWriter { public: /// \brief Creates a dataset writer /// /// Will fail if basename_template is invalid or if there is existing data and /// existing_data_behavior is kError /// - /// \param nursery an enclosing nursery which will contain the dataset write /// \param write_options options to control how the data should be written /// \param max_rows_queued max # of rows allowed to be queued before the dataset_writer /// will ask for backpressure - static Result>> - Make(util::Nursery* nursery, FileSystemDatasetWriteOptions write_options, - uint64_t max_rows_queued = kDefaultDatasetWriterMaxRowsQueued); + static Result> Make( + FileSystemDatasetWriteOptions write_options, + uint64_t max_rows_queued = kDefaultDatasetWriterMaxRowsQueued); ~DatasetWriter(); @@ -81,14 +80,15 @@ class ARROW_EXPORT DatasetWriter : public util::AsyncCloseablePimpl { Future<> WriteRecordBatch(std::shared_ptr batch, const std::string& directory); + /// Finish all pending writes and close any open files + Future<> Finish(); + protected: DatasetWriter(FileSystemDatasetWriteOptions write_options, uint64_t max_rows_queued = kDefaultDatasetWriterMaxRowsQueued); class DatasetWriterImpl; - std::unique_ptr impl_; - - friend util::Nursery; + std::unique_ptr> impl_; }; } // namespace dataset diff --git a/cpp/src/arrow/dataset/dataset_writer_test.cc b/cpp/src/arrow/dataset/dataset_writer_test.cc index 0bb78631b59..a834187732e 100644 --- a/cpp/src/arrow/dataset/dataset_writer_test.cc +++ b/cpp/src/arrow/dataset/dataset_writer_test.cc @@ -26,7 +26,6 @@ #include "arrow/filesystem/test_util.h" #include "arrow/testing/future_util.h" #include "arrow/testing/gtest_util.h" -#include "arrow/util/async_nursery.h" #include "arrow/util/optional.h" #include "gtest/gtest.h" @@ -94,7 +93,7 @@ class DatasetWriterTestFixture : public testing::Test { } void AssertVisited(const std::vector& actual_paths, - std::string expected_path) { + const std::string& expected_path) { std::vector::const_iterator found = std::find(actual_paths.begin(), actual_paths.end(), expected_path); ASSERT_NE(found, actual_paths.end()) @@ -118,14 +117,14 @@ class DatasetWriterTestFixture : public testing::Test { } } - void AssertNotFiles(const std::vector expected_non_files) { + void AssertNotFiles(const std::vector& expected_non_files) { for (const auto& expected_non_file : expected_non_files) { util::optional file = FindFile(expected_non_file); ASSERT_FALSE(file.has_value()); } } - void AssertEmptyFiles(const std::vector expected_empty_files) { + void AssertEmptyFiles(const std::vector& expected_empty_files) { for (const auto& expected_empty_file : expected_empty_files) { util::optional file = FindFile(expected_empty_file); ASSERT_TRUE(file.has_value()); @@ -142,53 +141,49 @@ class DatasetWriterTestFixture : public testing::Test { }; TEST_F(DatasetWriterTestFixture, Basic) { - ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { - EXPECT_OK_AND_ASSIGN(auto dataset_writer, - DatasetWriter::Make(nursery, write_options_)); - Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), ""); - AssertFinished(queue_fut); - })); + EXPECT_OK_AND_ASSIGN(auto dataset_writer, DatasetWriter::Make(write_options_)); + Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), ""); + AssertFinished(queue_fut); + ASSERT_FINISHES_OK(dataset_writer->Finish()); AssertFiles({{"testdir/part-0.arrow", 0, 100}}); } -TEST_F(DatasetWriterTestFixture, MaxRows) { +TEST_F(DatasetWriterTestFixture, MaxRowsOneWrite) { write_options_.max_rows_per_file = 10; - ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { - EXPECT_OK_AND_ASSIGN(auto dataset_writer, - DatasetWriter::Make(nursery, write_options_)); - Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(35), ""); - AssertFinished(queue_fut); - })); + EXPECT_OK_AND_ASSIGN(auto dataset_writer, DatasetWriter::Make(write_options_)); + Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(35), ""); + AssertFinished(queue_fut); + ASSERT_FINISHES_OK(dataset_writer->Finish()); AssertFiles({{"testdir/part-0.arrow", 0, 10}, {"testdir/part-1.arrow", 10, 20}, {"testdir/part-2.arrow", 20, 30}, {"testdir/part-3.arrow", 30, 35}}); - ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { - std::shared_ptr dataset_writer = - nursery->MakeSharedCloseable(write_options_); - ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); - ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); - ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); - ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); - ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); - ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); - })); +} + +TEST_F(DatasetWriterTestFixture, MaxRowsManyWrites) { + write_options_.max_rows_per_file = 10; + EXPECT_OK_AND_ASSIGN(auto dataset_writer, DatasetWriter::Make(write_options_)); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); + ASSERT_FINISHES_OK(dataset_writer->Finish()); AssertFiles({{"testdir/part-0.arrow", 0, 10}, {"testdir/part-1.arrow", 10, 8}}); } TEST_F(DatasetWriterTestFixture, ConcurrentWritesSameFile) { auto gated_fs = UseGatedFs(); - ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { - EXPECT_OK_AND_ASSIGN(auto dataset_writer, - DatasetWriter::Make(nursery, write_options_)); - for (int i = 0; i < 10; i++) { - Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(10), ""); - AssertFinished(queue_fut); - ASSERT_FINISHES_OK(queue_fut); - } - ASSERT_OK(gated_fs->WaitForOpenOutputStream(1)); - ASSERT_OK(gated_fs->UnlockOpenOutputStream(1)); - })); + EXPECT_OK_AND_ASSIGN(auto dataset_writer, DatasetWriter::Make(write_options_)); + for (int i = 0; i < 10; i++) { + Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(10), ""); + AssertFinished(queue_fut); + ASSERT_FINISHES_OK(queue_fut); + } + ASSERT_OK(gated_fs->WaitForOpenOutputStream(1)); + ASSERT_OK(gated_fs->UnlockOpenOutputStream(1)); + ASSERT_FINISHES_OK(dataset_writer->Finish()); AssertFiles({{"testdir/part-0.arrow", 0, 100}}); } @@ -197,47 +192,42 @@ TEST_F(DatasetWriterTestFixture, ConcurrentWritesDifferentFiles) { constexpr int NBATCHES = 6; auto gated_fs = UseGatedFs(); std::vector expected_files; - ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { - EXPECT_OK_AND_ASSIGN(auto dataset_writer, - DatasetWriter::Make(nursery, write_options_)); - for (int i = 0; i < NBATCHES; i++) { - std::string i_str = std::to_string(i); - expected_files.push_back(ExpectedFile{"testdir/part" + i_str + "/part-0.arrow", - static_cast(i) * 10, - (static_cast(i + 1) * 10)}); - Future<> queue_fut = - dataset_writer->WriteRecordBatch(MakeBatch(10), "part" + i_str); - AssertFinished(queue_fut); - ASSERT_FINISHES_OK(queue_fut); - } - ASSERT_OK(gated_fs->WaitForOpenOutputStream(NBATCHES)); - ASSERT_OK(gated_fs->UnlockOpenOutputStream(NBATCHES)); - })); + EXPECT_OK_AND_ASSIGN(auto dataset_writer, DatasetWriter::Make(write_options_)); + for (int i = 0; i < NBATCHES; i++) { + std::string i_str = std::to_string(i); + expected_files.push_back(ExpectedFile{"testdir/part" + i_str + "/part-0.arrow", + static_cast(i) * 10, + (static_cast(i + 1) * 10)}); + Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(10), "part" + i_str); + AssertFinished(queue_fut); + ASSERT_FINISHES_OK(queue_fut); + } + ASSERT_OK(gated_fs->WaitForOpenOutputStream(NBATCHES)); + ASSERT_OK(gated_fs->UnlockOpenOutputStream(NBATCHES)); + ASSERT_FINISHES_OK(dataset_writer->Finish()); AssertFiles(expected_files); } TEST_F(DatasetWriterTestFixture, MaxOpenFiles) { auto gated_fs = UseGatedFs(); write_options_.max_open_files = 2; - ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { - EXPECT_OK_AND_ASSIGN(auto dataset_writer, - DatasetWriter::Make(nursery, write_options_)); + EXPECT_OK_AND_ASSIGN(auto dataset_writer, DatasetWriter::Make(write_options_)); - ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(10), "part0")); - ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(10), "part1")); - ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(10), "part0")); - Future<> fut = dataset_writer->WriteRecordBatch(MakeBatch(10), "part2"); - // Backpressure will be applied until an existing file can be evicted - AssertNotFinished(fut); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(10), "part0")); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(10), "part1")); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(10), "part0")); + Future<> fut = dataset_writer->WriteRecordBatch(MakeBatch(10), "part2"); + // Backpressure will be applied until an existing file can be evicted + AssertNotFinished(fut); - // Ungate the writes to relieve the pressure, testdir/part0 should be closed - ASSERT_OK(gated_fs->WaitForOpenOutputStream(2)); - ASSERT_OK(gated_fs->UnlockOpenOutputStream(4)); - ASSERT_FINISHES_OK(fut); + // Ungate the writes to relieve the pressure, testdir/part0 should be closed + ASSERT_OK(gated_fs->WaitForOpenOutputStream(2)); + ASSERT_OK(gated_fs->UnlockOpenOutputStream(4)); + ASSERT_FINISHES_OK(fut); - ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(10), "part0")); - ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(10), "part1")); - })); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(10), "part0")); + ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(10), "part1")); + ASSERT_FINISHES_OK(dataset_writer->Finish()); AssertFiles({{"testdir/part0/part-0.arrow", 0, 10}, {"testdir/part0/part-0.arrow", 20, 10}, {"testdir/part0/part-1.arrow", 40, 10}, @@ -255,12 +245,10 @@ TEST_F(DatasetWriterTestFixture, DeleteExistingData) { filesystem_ = std::dynamic_pointer_cast(fs); write_options_.filesystem = filesystem_; write_options_.existing_data_behavior = kDeleteMatchingPartitions; - ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { - EXPECT_OK_AND_ASSIGN(auto dataset_writer, - DatasetWriter::Make(nursery, write_options_)); - Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), ""); - AssertFinished(queue_fut); - })); + EXPECT_OK_AND_ASSIGN(auto dataset_writer, DatasetWriter::Make(write_options_)); + Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), ""); + AssertFinished(queue_fut); + ASSERT_FINISHES_OK(dataset_writer->Finish()); AssertFiles({{"testdir/part-0.arrow", 0, 100}}); AssertNotFiles({"testdir/part-5.arrow", "testdir/blah.txt"}); } @@ -275,12 +263,10 @@ TEST_F(DatasetWriterTestFixture, PartitionedDeleteExistingData) { filesystem_ = std::dynamic_pointer_cast(fs); write_options_.filesystem = filesystem_; write_options_.existing_data_behavior = kDeleteMatchingPartitions; - ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { - EXPECT_OK_AND_ASSIGN(auto dataset_writer, - DatasetWriter::Make(nursery, write_options_)); - Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), "part0"); - AssertFinished(queue_fut); - })); + EXPECT_OK_AND_ASSIGN(auto dataset_writer, DatasetWriter::Make(write_options_)); + Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), "part0"); + AssertFinished(queue_fut); + ASSERT_FINISHES_OK(dataset_writer->Finish()); AssertFiles({{"testdir/part0/part-0.arrow", 0, 100}}); AssertNotFiles({"testdir/part0/foo.arrow"}); AssertEmptyFiles({"testdir/part1/bar.arrow"}); @@ -296,12 +282,10 @@ TEST_F(DatasetWriterTestFixture, LeaveExistingData) { filesystem_ = std::dynamic_pointer_cast(fs); write_options_.filesystem = filesystem_; write_options_.existing_data_behavior = kOverwriteOrIgnore; - ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { - EXPECT_OK_AND_ASSIGN(auto dataset_writer, - DatasetWriter::Make(nursery, write_options_)); - Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), ""); - AssertFinished(queue_fut); - })); + EXPECT_OK_AND_ASSIGN(auto dataset_writer, DatasetWriter::Make(write_options_)); + Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), ""); + AssertFinished(queue_fut); + ASSERT_FINISHES_OK(dataset_writer->Finish()); AssertFiles({{"testdir/part-0.arrow", 0, 100}}); AssertEmptyFiles({"testdir/part-5.arrow", "testdir/blah.txt"}); } @@ -315,9 +299,7 @@ TEST_F(DatasetWriterTestFixture, ErrOnExistingData) { fs::File("testdir/part-5.arrow"), fs::File("testdir/blah.txt")})); filesystem_ = std::dynamic_pointer_cast(fs); write_options_.filesystem = filesystem_; - ASSERT_OK(util::Nursery::RunInNursery([&](util::Nursery* nursery) { - ASSERT_RAISES(Invalid, DatasetWriter::Make(nursery, write_options_)); - })); + ASSERT_RAISES(Invalid, DatasetWriter::Make(write_options_)); AssertEmptyFiles({"testdir/part-0.arrow", "testdir/part-5.arrow", "testdir/blah.txt"}); } diff --git a/cpp/src/arrow/dataset/file_base.cc b/cpp/src/arrow/dataset/file_base.cc index d66f6e7413a..90fa48a783e 100644 --- a/cpp/src/arrow/dataset/file_base.cc +++ b/cpp/src/arrow/dataset/file_base.cc @@ -32,7 +32,6 @@ #include "arrow/io/compressed.h" #include "arrow/io/interfaces.h" #include "arrow/io/memory.h" -#include "arrow/util/async_nursery.h" #include "arrow/util/compression.h" #include "arrow/util/iterator.h" #include "arrow/util/macros.h" @@ -365,31 +364,28 @@ Future<> WriteNextBatch(DatasetWriter* dataset_writer, TaggedRecordBatch batch, Status FileSystemDataset::Write(const FileSystemDatasetWriteOptions& write_options, std::shared_ptr scanner) { - std::function task = [&](util::Nursery* nursery) { - ARROW_ASSIGN_OR_RAISE(auto batch_gen, scanner->ScanBatchesAsync()); - ARROW_ASSIGN_OR_RAISE(auto dataset_writer, - DatasetWriter::Make(nursery, write_options)); - - AsyncGenerator> queued_batch_gen = - [batch_gen, &dataset_writer, &write_options]() -> Future> { - Future next_batch_fut = batch_gen(); - return next_batch_fut.Then( - [&dataset_writer, &write_options](const TaggedRecordBatch& batch) { - if (IsIterationEnd(batch)) { - return AsyncGeneratorEnd>(); - } - return WriteNextBatch(dataset_writer.get(), batch, write_options).Then([] { - return std::make_shared(0); - }); + ARROW_ASSIGN_OR_RAISE(auto batch_gen, scanner->ScanBatchesAsync()); + ARROW_ASSIGN_OR_RAISE(auto dataset_writer, DatasetWriter::Make(write_options)); + + AsyncGenerator> queued_batch_gen = + [batch_gen, &dataset_writer, &write_options]() -> Future> { + Future next_batch_fut = batch_gen(); + return next_batch_fut.Then( + [&dataset_writer, &write_options](const TaggedRecordBatch& batch) { + if (IsIterationEnd(batch)) { + return AsyncGeneratorEnd>(); + } + return WriteNextBatch(dataset_writer.get(), batch, write_options).Then([] { + return std::make_shared(0); }); - }; - Future<> queue_fut = - VisitAsyncGenerator(std::move(queued_batch_gen), - [&](const std::shared_ptr&) { return Status::OK(); }); - - return queue_fut.status(); + }); }; - return util::Nursery::RunInNursery(std::move(task)); + Future<> queue_fut = + VisitAsyncGenerator(std::move(queued_batch_gen), + [&](const std::shared_ptr&) { return Status::OK(); }); + + ARROW_RETURN_NOT_OK(queue_fut.status()); + return dataset_writer->Finish().status(); } } // namespace dataset From e45ecf0a20c7f049ac7362e918b3cad517a4f261 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Thu, 9 Sep 2021 16:35:01 -1000 Subject: [PATCH 03/26] ARROW-13542: Rebased onto async utilities merged into master & addressed PR comments by simplifying the throttling mechanism --- cpp/src/arrow/dataset/dataset_writer.cc | 177 ++++++++---------------- 1 file changed, 61 insertions(+), 116 deletions(-) diff --git a/cpp/src/arrow/dataset/dataset_writer.cc b/cpp/src/arrow/dataset/dataset_writer.cc index 5feb55ba6d4..1deab466b16 100644 --- a/cpp/src/arrow/dataset/dataset_writer.cc +++ b/cpp/src/arrow/dataset/dataset_writer.cc @@ -36,97 +36,49 @@ namespace { constexpr util::string_view kIntegerToken = "{i}"; -class DatasetWriterStatistics { +class Throttle { public: - DatasetWriterStatistics(uint64_t max_rows_in_flight, uint32_t max_files_in_flight) - : max_rows_in_flight_(max_rows_in_flight), - max_files_in_flight_(max_files_in_flight) {} + explicit Throttle(uint64_t max_value) : max_value_(max_value) {} - bool CanWrite(const std::shared_ptr& record_batch, - const std::string& filename) { - std::lock_guard lg(mutex_); - uint64_t rows = record_batch->num_rows(); - DCHECK_LT(rows, max_rows_in_flight_); + bool Unthrottled() const { return max_value_ <= 0; } - if (rows_in_flight_ + rows > max_rows_in_flight_) { - rows_in_waiting_ = rows; - backpressure_ = Future<>::Make(); - return false; + Future<> Acquire(uint64_t values) { + if (Unthrottled()) { + return Future<>::MakeFinished(); } - return true; - } - - bool CanOpenFile() { std::lock_guard lg(mutex_); - if (files_in_flight_ == max_files_in_flight_) { - waiting_on_file_ = true; + if (values + current_value_ > max_value_) { + in_waiting_ = values; backpressure_ = Future<>::Make(); - return false; + } else { + current_value_ += values; } - return true; - } - - void RecordWriteStart(uint64_t num_rows) { - std::lock_guard lg(mutex_); - rows_in_flight_ += num_rows; - } - - void RecordFileStart() { - std::lock_guard lg(mutex_); - files_in_flight_++; - } - - void RecordFileFinished() { - std::unique_lock lk(mutex_); - files_in_flight_--; - FreeBackpressureIfPossible(std::move(lk)); - } - - void RecordWriteFinish(uint64_t num_rows) { - std::unique_lock lk(mutex_); - rows_in_flight_ -= num_rows; - FreeBackpressureIfPossible(std::move(lk)); - } - - Future<> backpressure() { - std::lock_guard lg(mutex_); return backpressure_; } - private: - void FreeBackpressureIfPossible(std::unique_lock&& lk) { - if (waiting_on_file_) { - if (files_in_flight_ < max_files_in_flight_) { - waiting_on_file_ = false; - } + void Release(uint64_t values) { + if (Unthrottled()) { + return; } - - bool waiting_on_rows = true; - if (rows_in_flight_ > 0) { - if (rows_in_flight_ + rows_in_waiting_ < max_rows_in_flight_) { - rows_in_waiting_ = 0; - waiting_on_rows = false; + Future<> to_complete; + { + std::lock_guard lg(mutex_); + current_value_ -= values; + if (in_waiting_ > 0 && in_waiting_ + current_value_ <= max_value_) { + in_waiting_ = 0; + to_complete = backpressure_; } - } else { - waiting_on_rows = false; } - - if (backpressure_.is_valid() && !waiting_on_rows && !waiting_on_file_) { - Future<> old_backpressure = backpressure_; - backpressure_ = Future<>(); - lk.unlock(); - old_backpressure.MarkFinished(); + if (to_complete.is_valid()) { + to_complete.MarkFinished(); } } - uint64_t max_rows_in_flight_; - uint32_t max_files_in_flight_; - - Future<> backpressure_; - uint64_t rows_in_flight_ = 0; - uint64_t rows_in_waiting_ = 0; - uint32_t files_in_flight_ = 0; - bool waiting_on_file_ = false; + private: + Future<> backpressure_ = Future<>::MakeFinished(); + uint64_t max_value_; + uint64_t in_waiting_ = 0; + uint64_t current_value_ = 0; std::mutex mutex_; }; @@ -263,12 +215,11 @@ class DatasetWriterDirectoryQueue : public util::AsyncDestroyable { public: DatasetWriterDirectoryQueue(std::string directory, std::shared_ptr schema, const FileSystemDatasetWriteOptions& write_options, - DatasetWriterStatistics* write_statistics, - std::mutex* visitors_mutex) + Throttle* open_files_throttle, std::mutex* visitors_mutex) : directory_(std::move(directory)), schema_(std::move(schema)), write_options_(write_options), - write_statistics_(write_statistics), + open_files_throttle_(open_files_throttle), visitors_mutex_(visitors_mutex) {} Result> NextWritableChunk( @@ -328,7 +279,6 @@ class DatasetWriterDirectoryQueue : public util::AsyncDestroyable { Result> OpenFileQueue( const std::string& filename) { - write_statistics_->RecordFileStart(); Future> file_writer_fut = init_future_.Then([this, filename] { ::arrow::internal::Executor* io_executor = @@ -338,12 +288,11 @@ class DatasetWriterDirectoryQueue : public util::AsyncDestroyable { }); auto file_queue = util::MakeSharedAsync( file_writer_fut, write_options_, visitors_mutex_); - RETURN_NOT_OK(task_group_.AddTask(file_queue->on_closed().Then( - [this] { write_statistics_->RecordFileFinished(); }))); + RETURN_NOT_OK(task_group_.AddTask( + file_queue->on_closed().Then([this] { open_files_throttle_->Release(1); }))); return file_queue; } - const std::string& current_filename() const { return current_filename_; } uint64_t rows_written() const { return rows_written_; } void PrepareDirectory() { @@ -363,11 +312,10 @@ class DatasetWriterDirectoryQueue : public util::AsyncDestroyable { static Result>> Make(util::AsyncTaskGroup* task_group, - const FileSystemDatasetWriteOptions& write_options, - DatasetWriterStatistics* write_statistics, std::shared_ptr schema, - std::string dir, std::mutex* visitors_mutex) { + const FileSystemDatasetWriteOptions& write_options, Throttle* open_files_throttle, + std::shared_ptr schema, std::string dir, std::mutex* visitors_mutex) { auto dir_queue = util::MakeUniqueAsync( - std::move(dir), std::move(schema), write_options, write_statistics, + std::move(dir), std::move(schema), write_options, open_files_throttle, visitors_mutex); RETURN_NOT_OK(task_group->AddTask(dir_queue->on_closed())); dir_queue->PrepareDirectory(); @@ -385,7 +333,7 @@ class DatasetWriterDirectoryQueue : public util::AsyncDestroyable { std::string directory_; std::shared_ptr schema_; const FileSystemDatasetWriteOptions& write_options_; - DatasetWriterStatistics* write_statistics_; + Throttle* open_files_throttle_; std::mutex* visitors_mutex_; Future<> init_future_; std::string current_filename_; @@ -431,7 +379,8 @@ class DatasetWriter::DatasetWriterImpl : public util::AsyncDestroyable { public: DatasetWriterImpl(FileSystemDatasetWriteOptions write_options, uint64_t max_rows_queued) : write_options_(std::move(write_options)), - statistics_(max_rows_queued, write_options.max_open_files) {} + rows_in_flight_throttle_(max_rows_queued), + open_files_throttle_(write_options.max_open_files) {} Future<> WriteRecordBatch(std::shared_ptr batch, const std::string& directory) { @@ -465,13 +414,13 @@ class DatasetWriter::DatasetWriterImpl : public util::AsyncDestroyable { auto dir_queue_itr, ::arrow::internal::GetOrInsertGenerated( &directory_queues_, directory, [this, &batch](const std::string& dir) { - return DatasetWriterDirectoryQueue::Make(&task_group_, write_options_, - &statistics_, batch->schema(), dir, - &visitors_mutex_); + return DatasetWriterDirectoryQueue::Make( + &task_group_, write_options_, &open_files_throttle_, batch->schema(), + dir, &visitors_mutex_); })); std::shared_ptr dir_queue = dir_queue_itr->second; std::vector> scheduled_writes; - bool hit_backpressure = false; + Future<> backpressure; while (batch) { // Keep opening new files until batch is done. std::shared_ptr remainder; @@ -479,21 +428,22 @@ class DatasetWriter::DatasetWriterImpl : public util::AsyncDestroyable { ARROW_ASSIGN_OR_RAISE(auto next_chunk, dir_queue->NextWritableChunk( batch, &remainder, &will_open_file)); - if (statistics_.CanWrite(next_chunk, dir_queue->current_filename()) && - (!will_open_file || statistics_.CanOpenFile())) { - statistics_.RecordWriteStart(next_chunk->num_rows()); - scheduled_writes.push_back(dir_queue->StartWrite(next_chunk)); - batch = std::move(remainder); - if (batch) { - ARROW_RETURN_NOT_OK(dir_queue->FinishCurrentFile()); - } - } else { - if (!statistics_.CanOpenFile()) { - ARROW_RETURN_NOT_OK(CloseLargestFile()); - } - hit_backpressure = true; + backpressure = rows_in_flight_throttle_.Acquire(next_chunk->num_rows()); + if (!backpressure.is_finished()) { break; } + if (will_open_file) { + backpressure = open_files_throttle_.Acquire(1); + if (!backpressure.is_finished()) { + RETURN_NOT_OK(CloseLargestFile()); + break; + } + } + scheduled_writes.push_back(dir_queue->StartWrite(next_chunk)); + batch = std::move(remainder); + if (batch) { + RETURN_NOT_OK(dir_queue->FinishCurrentFile()); + } } for (auto& scheduled_write : scheduled_writes) { @@ -502,19 +452,13 @@ class DatasetWriter::DatasetWriterImpl : public util::AsyncDestroyable { RETURN_NOT_OK(CheckError()); RETURN_NOT_OK(task_group_.AddTask(scheduled_write.Then( [this](const WriteTask& write) { - statistics_.RecordWriteFinish(write.num_rows); + rows_in_flight_throttle_.Release(write.num_rows); }, [this](const Status& err) { SetError(err); }))); } - if (hit_backpressure) { - Future<> maybe_backpressure = statistics_.backpressure(); - // It's possible the backpressure was relieved since we last checked - if (maybe_backpressure.is_valid()) { - return maybe_backpressure.Then( - [this, batch, directory] { return DoWriteRecordBatch(batch, directory); }); - } else { - return DoWriteRecordBatch(batch, directory); - } + if (batch) { + return backpressure.Then( + [this, batch, directory] { return DoWriteRecordBatch(batch, directory); }); } return Future<>::MakeFinished(); } @@ -536,7 +480,8 @@ class DatasetWriter::DatasetWriterImpl : public util::AsyncDestroyable { util::AsyncTaskGroup task_group_; FileSystemDatasetWriteOptions write_options_; - DatasetWriterStatistics statistics_; + Throttle rows_in_flight_throttle_; + Throttle open_files_throttle_; std::unordered_map> directory_queues_; std::mutex mutex_; From c9d91ca6ef6891cef5ab39010d76b085441a5ded Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Fri, 10 Sep 2021 11:40:32 -1000 Subject: [PATCH 04/26] ARROW-13542: Fixed python bindings (defualt to use_async=True for write dataset scanners) and fixed python tests to expect new file ordering --- python/pyarrow/dataset.py | 10 +++------ python/pyarrow/tests/test_dataset.py | 32 ++++++++++------------------ 2 files changed, 14 insertions(+), 28 deletions(-) diff --git a/python/pyarrow/dataset.py b/python/pyarrow/dataset.py index 3ab8dada86e..70aeb150b1b 100644 --- a/python/pyarrow/dataset.py +++ b/python/pyarrow/dataset.py @@ -736,7 +736,7 @@ def _ensure_write_partitioning(part, schema, flavor): def write_dataset(data, base_dir, basename_template=None, format=None, partitioning=None, partitioning_flavor=None, schema=None, filesystem=None, file_options=None, use_threads=True, - use_async=False, max_partitions=None, file_visitor=None): + max_partitions=None, file_visitor=None): """ Write a dataset to a given format and partitioning. @@ -777,10 +777,6 @@ def write_dataset(data, base_dir, basename_template=None, format=None, use_threads : bool, default True Write files in parallel. If enabled, then maximum parallelism will be used determined by the number of available CPU cores. - use_async : bool, default False - If enabled, an async scanner will be used that should offer - better performance with high-latency/highly-parallel filesystems - (e.g. S3) max_partitions : int, default 1024 Maximum number of partitions any batch may be written into. file_visitor : Function @@ -812,7 +808,7 @@ def file_visitor(written_file): schema = schema or data.schema data = InMemoryDataset(data, schema=schema) elif isinstance(data, pa.ipc.RecordBatchReader) or _is_iterable(data): - data = Scanner.from_batches(data, schema=schema) + data = Scanner.from_batches(data, schema=schema, use_async=True) schema = None elif not isinstance(data, (Dataset, Scanner)): raise ValueError( @@ -854,7 +850,7 @@ def file_visitor(written_file): filesystem, base_dir = _resolve_filesystem_and_path(base_dir, filesystem) if isinstance(data, Dataset): - scanner = data.scanner(use_threads=use_threads, use_async=use_async) + scanner = data.scanner(use_threads=use_threads, use_async=True) else: # scanner was passed directly by the user, in which case a schema # cannot be passed diff --git a/python/pyarrow/tests/test_dataset.py b/python/pyarrow/tests/test_dataset.py index 900cd650db2..e121c6b9893 100644 --- a/python/pyarrow/tests/test_dataset.py +++ b/python/pyarrow/tests/test_dataset.py @@ -3296,7 +3296,7 @@ def test_write_dataset_partitioned(tempdir): target = tempdir / 'partitioned-hive-target' expected_paths = [ target / "part=a", target / "part=a" / "part-0.feather", - target / "part=b", target / "part=b" / "part-1.feather" + target / "part=b", target / "part=b" / "part-0.feather" ] partitioning_schema = ds.partitioning( pa.schema([("part", pa.string())]), flavor="hive") @@ -3308,7 +3308,7 @@ def test_write_dataset_partitioned(tempdir): target = tempdir / 'partitioned-dir-target' expected_paths = [ target / "a", target / "a" / "part-0.feather", - target / "b", target / "b" / "part-1.feather" + target / "b", target / "b" / "part-0.feather" ] partitioning_schema = ds.partitioning( pa.schema([("part", pa.string())])) @@ -3400,7 +3400,7 @@ def test_write_dataset_partitioned_dict(tempdir): target = tempdir / 'partitioned-dir-target' expected_paths = [ target / "a", target / "a" / "part-0.feather", - target / "b", target / "b" / "part-1.feather" + target / "b", target / "b" / "part-0.feather" ] partitioning = ds.partitioning(pa.schema([ dataset.schema.field('part')]), @@ -3434,18 +3434,12 @@ def file_visitor(written_file): use_threads=True, file_visitor=file_visitor ) - # Since it is a multi-threaded write there is no way to know which - # directory gets part-0 and which gets part-1 - expected_paths_a = { + expected_paths = { target1 / 'part=a' / 'part-0.feather', - target1 / 'part=b' / 'part-1.feather' - } - expected_paths_b = { - target1 / 'part=a' / 'part-1.feather', target1 / 'part=b' / 'part-0.feather' } paths_written_set = set(map(pathlib.Path, paths_written)) - assert paths_written_set in [expected_paths_a, expected_paths_b] + assert paths_written_set == expected_paths target2 = tempdir / 'partitioned2' ds.write_dataset( @@ -3480,7 +3474,7 @@ def test_write_table(tempdir): base_dir = tempdir / 'partitioned' expected_paths = [ base_dir / "part=a", base_dir / "part=a" / "dat_0.arrow", - base_dir / "part=b", base_dir / "part=b" / "dat_1.arrow" + base_dir / "part=b", base_dir / "part=b" / "dat_0.arrow" ] visited_paths = [] @@ -3561,10 +3555,10 @@ def test_write_iterable(tempdir): def test_write_scanner(tempdir, dataset_reader): - if dataset_reader.use_async: + if not dataset_reader.use_async: pytest.skip( - ('ARROW-12803: Write dataset with scanner does not' - ' support async scan')) + ('ARROW-13338: Write dataset with scanner does not' + ' support synchronous scan')) table = pa.table([ pa.array(range(20)), pa.array(np.random.randn(20)), @@ -3722,16 +3716,12 @@ def file_visitor(written_file): # Since it is a multi-threaded write there is no way to know which # directory gets part-0 and which gets part-1 - expected_paths_a = { + expected_paths = { root_path / 'part=a' / 'part-0.parquet', - root_path / 'part=b' / 'part-1.parquet' - } - expected_paths_b = { - root_path / 'part=a' / 'part-1.parquet', root_path / 'part=b' / 'part-0.parquet' } paths_written_set = set(map(pathlib.Path, paths_written)) - assert paths_written_set in [expected_paths_a, expected_paths_b] + assert paths_written_set == expected_paths assert sample_metadata is not None assert sample_metadata.num_columns == 2 From 5fcf0aa3327e4bf00820cc5a0fc873c2241afdca Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Fri, 10 Sep 2021 12:03:41 -1000 Subject: [PATCH 05/26] ARROW-13542: Added R support by changing write_dataset to create an async scanner by default --- r/R/dataset-write.R | 2 +- r/tests/testthat/test-metadata.R | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/r/R/dataset-write.R b/r/R/dataset-write.R index b17c5f39ba8..1253445de19 100644 --- a/r/R/dataset-write.R +++ b/r/R/dataset-write.R @@ -111,7 +111,7 @@ write_dataset <- function(dataset, dataset <- dplyr::ungroup(dataset) } - scanner <- Scanner$create(dataset) + scanner <- Scanner$create(dataset, use_async=TRUE) if (!inherits(partitioning, "Partitioning")) { partition_schema <- scanner$schema[partitioning] if (isTRUE(hive_style)) { diff --git a/r/tests/testthat/test-metadata.R b/r/tests/testthat/test-metadata.R index c8e3244cf0f..34c6b536f87 100644 --- a/r/tests/testthat/test-metadata.R +++ b/r/tests/testthat/test-metadata.R @@ -246,6 +246,9 @@ test_that("metadata of list elements (ARROW-10386)", { "Row-level metadata is not compatible with datasets and will be discarded" ) + # Reset directory as previous write will have created some files and the defualt + # behavior is to error on existing + dst_dir <- make_temp_dir() # but we need to write a dataset with row-level metadata to make sure when # reading ones that have been written with them we warn appropriately fake_func_name <- write_dataset From 96b689be9989460e87f6225f1fc5e7fd88da59ae Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Fri, 10 Sep 2021 13:44:04 -1000 Subject: [PATCH 06/26] ARROW-13542: Fixed race condition in counting semaphore test. Decreased num iterations to speed up CI --- cpp/src/arrow/dataset/dataset_writer.h | 2 +- cpp/src/arrow/util/counting_semaphore_test.cc | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/cpp/src/arrow/dataset/dataset_writer.h b/cpp/src/arrow/dataset/dataset_writer.h index 3b9e1c88b35..21c9fad4491 100644 --- a/cpp/src/arrow/dataset/dataset_writer.h +++ b/cpp/src/arrow/dataset/dataset_writer.h @@ -37,7 +37,7 @@ constexpr uint64_t kDefaultDatasetWriterMaxRowsQueued = 64 * 1024 * 1024; /// /// The dataset writer enforces its own back pressure based on the # of rows (as opposed /// to # of batches which is how it is typically enforced elsewhere) and # of files. -class ARROW_EXPORT DatasetWriter { +class ARROW_DS_EXPORT DatasetWriter { public: /// \brief Creates a dataset writer /// diff --git a/cpp/src/arrow/util/counting_semaphore_test.cc b/cpp/src/arrow/util/counting_semaphore_test.cc index e5369525fc9..a5fa9f6bde8 100644 --- a/cpp/src/arrow/util/counting_semaphore_test.cc +++ b/cpp/src/arrow/util/counting_semaphore_test.cc @@ -64,7 +64,7 @@ TEST(CountingSemaphore, CloseAborts) { } TEST(CountingSemaphore, Stress) { - constexpr uint32_t NTHREADS = 100; + constexpr uint32_t NTHREADS = 10; CountingSemaphore semaphore; std::vector max_allowed_cases = {1, 3}; std::atomic count{0}; @@ -77,7 +77,6 @@ TEST(CountingSemaphore, Stress) { ASSERT_OK(semaphore.Acquire(1)); uint32_t last_count = count.fetch_add(1); if (last_count >= max_allowed) { - std::cout << last_count << std::endl; max_exceeded.store(true); } SleepABit(); @@ -85,11 +84,12 @@ TEST(CountingSemaphore, Stress) { ASSERT_OK(semaphore.Release(1)); }); } + for (auto& thread : threads) { + thread.join(); + } + threads.clear(); ASSERT_OK(semaphore.Acquire(max_allowed)); } - for (auto& thread : threads) { - thread.join(); - } ASSERT_OK(semaphore.Close()); ASSERT_FALSE(max_exceeded.load()); } From 38f7960383d20ade8bbead614f54d369bf560187 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Fri, 10 Sep 2021 14:50:39 -1000 Subject: [PATCH 07/26] ARROW-13542: Trying to fix more tests/build errors --- cpp/src/arrow/dataset/dataset_writer.cc | 3 ++- r/R/dataset.R | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/dataset/dataset_writer.cc b/cpp/src/arrow/dataset/dataset_writer.cc index 1deab466b16..d9bc2890a2a 100644 --- a/cpp/src/arrow/dataset/dataset_writer.cc +++ b/cpp/src/arrow/dataset/dataset_writer.cc @@ -320,7 +320,8 @@ class DatasetWriterDirectoryQueue : public util::AsyncDestroyable { RETURN_NOT_OK(task_group->AddTask(dir_queue->on_closed())); dir_queue->PrepareDirectory(); ARROW_ASSIGN_OR_RAISE(dir_queue->current_filename_, dir_queue->GetNextFilename()); - return dir_queue; + // std::move required to make RTools 3.5 mingw compiler happy + return std::move(dir_queue); } Future<> DoDestroy() override { diff --git a/r/R/dataset.R b/r/R/dataset.R index 593d3f55141..970a07102fd 100644 --- a/r/R/dataset.R +++ b/r/R/dataset.R @@ -95,7 +95,7 @@ #' open_dataset(tf) #' #' # You can also supply a vector of paths -#' open_dataset(c(file.path(tf, "cyl=4/part-1.parquet"), file.path(tf, "cyl=8/part-2.parquet"))) +#' open_dataset(c(file.path(tf, "cyl=4/part-0.parquet"), file.path(tf, "cyl=8/part-0.parquet"))) #' #' ## You must specify the file format if using a format other than parquet. #' tf2 <- tempfile() From 32e94c1856e12b69802b1fd92ff9854c380befb3 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Mon, 13 Sep 2021 19:13:44 -1000 Subject: [PATCH 08/26] ARROW-13542: Found a bug in dataset_writer_test::DatasetWriterTestFixture.MaxOpenFiles --- cpp/src/arrow/dataset/dataset_writer_test.cc | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/dataset/dataset_writer_test.cc b/cpp/src/arrow/dataset/dataset_writer_test.cc index a834187732e..f37aebc5bc3 100644 --- a/cpp/src/arrow/dataset/dataset_writer_test.cc +++ b/cpp/src/arrow/dataset/dataset_writer_test.cc @@ -222,10 +222,12 @@ TEST_F(DatasetWriterTestFixture, MaxOpenFiles) { // Ungate the writes to relieve the pressure, testdir/part0 should be closed ASSERT_OK(gated_fs->WaitForOpenOutputStream(2)); - ASSERT_OK(gated_fs->UnlockOpenOutputStream(4)); + ASSERT_OK(gated_fs->UnlockOpenOutputStream(5)); ASSERT_FINISHES_OK(fut); ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(10), "part0")); + // Following call should resume existing write but, on slow test systems, the old + // write may have already been finished ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(10), "part1")); ASSERT_FINISHES_OK(dataset_writer->Finish()); AssertFiles({{"testdir/part0/part-0.arrow", 0, 10}, From 339d548859f697f20b9aab95c20042ec431384e7 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Mon, 13 Sep 2021 23:47:00 -1000 Subject: [PATCH 09/26] ARROW-13542: Fixed a bug in the async_util custom deleter that could be caused when using custom deleter types in standard containers --- cpp/src/arrow/util/async_util.h | 6 +++++- cpp/src/arrow/util/async_util_test.cc | 27 +++++++++++++++++++++++++++ 2 files changed, 32 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/util/async_util.h b/cpp/src/arrow/util/async_util.h index 31e5d09a86c..9b0efd9e030 100644 --- a/cpp/src/arrow/util/async_util.h +++ b/cpp/src/arrow/util/async_util.h @@ -28,7 +28,11 @@ namespace util { /// Custom deleter for AsyncDestroyable objects template struct DestroyingDeleter { - void operator()(T* p) { p->Destroy(); } + void operator()(T* p) { + if (p) { + p->Destroy(); + } + } }; /// An object which should be asynchronously closed before it is destroyed diff --git a/cpp/src/arrow/util/async_util_test.cc b/cpp/src/arrow/util/async_util_test.cc index f263ee548cf..9bae7977d45 100644 --- a/cpp/src/arrow/util/async_util_test.cc +++ b/cpp/src/arrow/util/async_util_test.cc @@ -63,6 +63,33 @@ TEST(AsyncDestroyable, MakeShared) { }); } +// The next four tests are corner cases but can sometimes occur when using these types +// in standard containers on certain versions of the compiler/cpplib. Basically we +// want to make sure our deleter is ok with null pointers. +TEST(AsyncDestroyable, DefaultUnique) { + std::unique_ptr> default_ptr; + default_ptr.reset(); +} + +TEST(AsyncDestroyable, NullUnique) { + std::unique_ptr> null_ptr( + nullptr); + null_ptr.reset(); +} + +TEST(AsyncDestroyable, NullShared) { + std::shared_ptr null_ptr(nullptr, + DestroyingDeleter()); + null_ptr.reset(); +} + +TEST(AsyncDestroyable, NullUniqueToShared) { + std::unique_ptr> null_ptr( + nullptr); + std::shared_ptr null_shared = std::move(null_ptr); + null_shared.reset(); +} + TEST(AsyncDestroyable, MakeUnique) { TestAsyncDestroyable([](Future<> gate, bool* destroyed) { return MakeUniqueAsync(gate, destroyed); From c01a3df1f01089ce59c3c283932acd7fc0dd840d Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 14 Sep 2021 00:20:17 -1000 Subject: [PATCH 10/26] ARROW-13542: Updated auto-generated R documentation --- r/man/open_dataset.Rd | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/r/man/open_dataset.Rd b/r/man/open_dataset.Rd index 59717863932..4d6b492e302 100644 --- a/r/man/open_dataset.Rd +++ b/r/man/open_dataset.Rd @@ -105,7 +105,7 @@ write_dataset(data, tf) open_dataset(tf) # You can also supply a vector of paths -open_dataset(c(file.path(tf, "cyl=4/part-1.parquet"), file.path(tf, "cyl=8/part-2.parquet"))) +open_dataset(c(file.path(tf, "cyl=4/part-0.parquet"), file.path(tf, "cyl=8/part-0.parquet"))) ## You must specify the file format if using a format other than parquet. tf2 <- tempfile() From 20067bcc91557f1a25892c68c32a6fe6e4711c66 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 14 Sep 2021 00:24:32 -1000 Subject: [PATCH 11/26] ARROW-13542: Attempt at switching ruby to use async scanner --- ruby/red-arrow-dataset/lib/arrow-dataset/arrow-table-savable.rb | 1 + testing | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/ruby/red-arrow-dataset/lib/arrow-dataset/arrow-table-savable.rb b/ruby/red-arrow-dataset/lib/arrow-dataset/arrow-table-savable.rb index f3896f04b92..30ad6c2927b 100644 --- a/ruby/red-arrow-dataset/lib/arrow-dataset/arrow-table-savable.rb +++ b/ruby/red-arrow-dataset/lib/arrow-dataset/arrow-table-savable.rb @@ -39,6 +39,7 @@ def save_to_uri options.base_name_template = File.basename(path) options.partitioning = Partitioning.resolve(@options[:partitioning]) scanner_builder = ScannerBuilder.new(@table) + scanner_builder.use_async(true) scanner = scanner_builder.finish FileSystemDataset.write_scanner(scanner, options) else diff --git a/testing b/testing index 2c29a733ac2..6d98243093c 160000 --- a/testing +++ b/testing @@ -1 +1 @@ -Subproject commit 2c29a733ac2c8492d5df3b74ea5ab1a32f892f60 +Subproject commit 6d98243093c0b36442da94de7010f3eacc2a9909 From ec49b38ae8bd0a5d9147c9dc0ebe335d2078c729 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 14 Sep 2021 08:32:56 -1000 Subject: [PATCH 12/26] ARROW-13542: Fixed c_glib test to use async scanner for dataset writes --- c_glib/test/dataset/test-file-system-dataset.rb | 1 + 1 file changed, 1 insertion(+) diff --git a/c_glib/test/dataset/test-file-system-dataset.rb b/c_glib/test/dataset/test-file-system-dataset.rb index 0e856b678f8..dfcf59658fd 100644 --- a/c_glib/test/dataset/test-file-system-dataset.rb +++ b/c_glib/test/dataset/test-file-system-dataset.rb @@ -60,6 +60,7 @@ def test_read_write count: build_int32_array([1, 10, 2, 3])) table_reader = Arrow::TableBatchReader.new(table) scanner_builder = ArrowDataset::ScannerBuilder.new(table_reader) + scanner.use_async(true) scanner = scanner_builder.finish options = ArrowDataset::FileSystemDatasetWriteOptions.new options.file_write_options = @format.default_write_options From cf9cb0fb4586e206b8a17f7b4caeec4ec97ed9be Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 14 Sep 2021 08:55:20 -1000 Subject: [PATCH 13/26] ARROW-13542: Need to call use_async on the scanner_builder and not the scanner --- c_glib/test/dataset/test-file-system-dataset.rb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/c_glib/test/dataset/test-file-system-dataset.rb b/c_glib/test/dataset/test-file-system-dataset.rb index dfcf59658fd..0e91d755db9 100644 --- a/c_glib/test/dataset/test-file-system-dataset.rb +++ b/c_glib/test/dataset/test-file-system-dataset.rb @@ -60,7 +60,7 @@ def test_read_write count: build_int32_array([1, 10, 2, 3])) table_reader = Arrow::TableBatchReader.new(table) scanner_builder = ArrowDataset::ScannerBuilder.new(table_reader) - scanner.use_async(true) + scanner_builder.use_async(true) scanner = scanner_builder.finish options = ArrowDataset::FileSystemDatasetWriteOptions.new options.file_write_options = @format.default_write_options From 20fadb506c80db8a308bb7f05fb2d1804a37140b Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 14 Sep 2021 13:42:52 -1000 Subject: [PATCH 14/26] ARROW-13542: Touched up Ruby API --- c_glib/arrow-dataset-glib/scanner.cpp | 19 +++++++++++++++++++ c_glib/arrow-dataset-glib/scanner.h | 6 ++++++ 2 files changed, 25 insertions(+) diff --git a/c_glib/arrow-dataset-glib/scanner.cpp b/c_glib/arrow-dataset-glib/scanner.cpp index ddd3fd88af7..eefd6c748b7 100644 --- a/c_glib/arrow-dataset-glib/scanner.cpp +++ b/c_glib/arrow-dataset-glib/scanner.cpp @@ -244,6 +244,25 @@ gadataset_scanner_builder_new_record_batch_reader( return gadataset_scanner_builder_new_raw(&arrow_scanner_builder); } +/** + * gadataset_scanner_builder_use_async: + * @builder: A #GADatasetScannerBuilder. + * @use_async: Use the asynchronous scanner + * @error: (nullable): Return location for a #GError or %NULL. + * + * Returns: void + * + * Since: 6.0.0 + */ +void +gadataset_scanner_builder_use_async(GADatasetScannerBuilder *builder, gboolean use_async, + GError **error) +{ + auto arrow_builder = gadataset_scanner_builder_get_raw(builder); + auto use_async_result = arrow_builder->UseAsync(use_async); + garrow::check(error, use_async_result, "[scanner-builder][use_async]"); +} + /** * gadataset_scanner_builder_finish: * @builder: A #GADatasetScannerBuilder. diff --git a/c_glib/arrow-dataset-glib/scanner.h b/c_glib/arrow-dataset-glib/scanner.h index ba7f9c6b7c3..4afec00c282 100644 --- a/c_glib/arrow-dataset-glib/scanner.h +++ b/c_glib/arrow-dataset-glib/scanner.h @@ -59,6 +59,12 @@ GARROW_AVAILABLE_IN_6_0 GADatasetScannerBuilder * gadataset_scanner_builder_new_record_batch_reader( GArrowRecordBatchReader *reader); + +GARROW_AVAILABLE_IN_6_0 +void +gadataset_scanner_builder_use_async( + GADatasetScannerBuilder *builder, gboolean use_async, GError **error); + GARROW_AVAILABLE_IN_5_0 GADatasetScanner * gadataset_scanner_builder_finish(GADatasetScannerBuilder *builder, From 0c83f28e2641f452e196450c51be86a5dabd2375 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 14 Sep 2021 14:26:20 -1000 Subject: [PATCH 15/26] ARROW-13542: Removing formatting-only changes --- cpp/src/arrow/filesystem/test_util.cc | 13 ++++--------- cpp/src/arrow/testing/gtest_util.h | 4 ++-- 2 files changed, 6 insertions(+), 11 deletions(-) diff --git a/cpp/src/arrow/filesystem/test_util.cc b/cpp/src/arrow/filesystem/test_util.cc index af7b5ebb7b0..da17447a4b2 100644 --- a/cpp/src/arrow/filesystem/test_util.cc +++ b/cpp/src/arrow/filesystem/test_util.cc @@ -15,22 +15,17 @@ // specific language governing permissions and limitations // under the License. -#include -#include - -// Can't import test_util.h before gtest.h because it depends on ASSERT_GE but clang -// wants the "related import" to come first so we need to disable here -// clang-format off -#include "arrow/filesystem/test_util.h" -// clang-format on - #include #include #include #include #include +#include +#include + #include "arrow/filesystem/mockfs.h" +#include "arrow/filesystem/test_util.h" #include "arrow/io/interfaces.h" #include "arrow/status.h" #include "arrow/testing/future_util.h" diff --git a/cpp/src/arrow/testing/gtest_util.h b/cpp/src/arrow/testing/gtest_util.h index d04efbf37da..e1baec50ae2 100644 --- a/cpp/src/arrow/testing/gtest_util.h +++ b/cpp/src/arrow/testing/gtest_util.h @@ -17,8 +17,6 @@ #pragma once -#include - #include #include #include @@ -30,6 +28,8 @@ #include #include +#include + #include "arrow/array/builder_binary.h" #include "arrow/array/builder_primitive.h" #include "arrow/array/builder_time.h" From 440589b0f38ae6972653c03a9155fc72881929aa Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 14 Sep 2021 14:28:26 -1000 Subject: [PATCH 16/26] ARROW-13542: Reverted changes to testing --- testing | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/testing b/testing index 6d98243093c..2c29a733ac2 160000 --- a/testing +++ b/testing @@ -1 +1 @@ -Subproject commit 6d98243093c0b36442da94de7010f3eacc2a9909 +Subproject commit 2c29a733ac2c8492d5df3b74ea5ab1a32f892f60 From 724d9226f0699fdee7667e403c86a7a5ce63e523 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 14 Sep 2021 15:39:08 -1000 Subject: [PATCH 17/26] ARROW-13542: Added verification of written file content to dataset writer tests --- cpp/src/arrow/dataset/dataset_writer_test.cc | 55 +++++++++++++++----- 1 file changed, 43 insertions(+), 12 deletions(-) diff --git a/cpp/src/arrow/dataset/dataset_writer_test.cc b/cpp/src/arrow/dataset/dataset_writer_test.cc index f37aebc5bc3..c1ade277ecb 100644 --- a/cpp/src/arrow/dataset/dataset_writer_test.cc +++ b/cpp/src/arrow/dataset/dataset_writer_test.cc @@ -24,6 +24,9 @@ #include "arrow/dataset/file_ipc.h" #include "arrow/filesystem/mockfs.h" #include "arrow/filesystem/test_util.h" +#include "arrow/io/memory.h" +#include "arrow/ipc/reader.h" +#include "arrow/table.h" #include "arrow/testing/future_util.h" #include "arrow/testing/gtest_util.h" #include "arrow/util/optional.h" @@ -73,16 +76,21 @@ class DatasetWriterTestFixture : public testing::Test { return fs; } - std::shared_ptr MakeBatch(uint64_t num_rows) { + std::shared_ptr MakeBatch(uint64_t start, uint64_t num_rows) { Int64Builder builder; - for (uint64_t i = counter_; i < counter_ + num_rows; i++) { - ARROW_EXPECT_OK(builder.Append(i)); + for (uint64_t i = 0; i < num_rows; i++) { + ARROW_EXPECT_OK(builder.Append(i + start)); } EXPECT_OK_AND_ASSIGN(std::shared_ptr arr, builder.Finish()); - counter_ += num_rows; return RecordBatch::Make(schema_, static_cast(num_rows), {std::move(arr)}); } + std::shared_ptr MakeBatch(uint64_t num_rows) { + std::shared_ptr batch = MakeBatch(counter_, num_rows); + counter_ += num_rows; + return batch; + } + util::optional FindFile(const std::string& filename) { for (const auto& mock_file : filesystem_->AllFiles()) { if (mock_file.full_path == filename) { @@ -100,7 +108,27 @@ class DatasetWriterTestFixture : public testing::Test { << "The file " << expected_path << " was not in the list of files visited"; } - void AssertFiles(const std::vector& expected_files) { + std::shared_ptr ReadAsBatch(util::string_view data) { + std::shared_ptr in_stream = + std::make_shared(data); + EXPECT_OK_AND_ASSIGN(std::shared_ptr reader, + ipc::RecordBatchFileReader::Open(std::move(in_stream))); + RecordBatchVector batches; + for (int i = 0; i < reader->num_record_batches(); i++) { + EXPECT_OK_AND_ASSIGN(std::shared_ptr next_batch, + reader->ReadRecordBatch(i)); + batches.push_back(next_batch); + } + EXPECT_OK_AND_ASSIGN(std::shared_ptr table, Table::FromRecordBatches(batches)); + EXPECT_OK_AND_ASSIGN(std::shared_ptr
combined_table, table->CombineChunks()); + EXPECT_OK_AND_ASSIGN(std::shared_ptr batch, + TableBatchReader(*combined_table).Next()); + return batch; + } + + void AssertFiles(const std::vector& expected_files, + bool verify_content = true) { + counter_ = 0; for (const auto& expected_file : expected_files) { util::optional written_file = FindFile(expected_file.filename); ASSERT_TRUE(written_file.has_value()) @@ -113,7 +141,10 @@ class DatasetWriterTestFixture : public testing::Test { SCOPED_TRACE("post_finish"); AssertVisited(post_finish_visited_, expected_file.filename); } - // FIXME Check contents + if (verify_content) { + AssertBatchesEqual(*MakeBatch(expected_file.start, expected_file.num_rows), + *ReadAsBatch(written_file->data)); + } } } @@ -155,9 +186,9 @@ TEST_F(DatasetWriterTestFixture, MaxRowsOneWrite) { AssertFinished(queue_fut); ASSERT_FINISHES_OK(dataset_writer->Finish()); AssertFiles({{"testdir/part-0.arrow", 0, 10}, - {"testdir/part-1.arrow", 10, 20}, - {"testdir/part-2.arrow", 20, 30}, - {"testdir/part-3.arrow", 30, 35}}); + {"testdir/part-1.arrow", 10, 10}, + {"testdir/part-2.arrow", 20, 10}, + {"testdir/part-3.arrow", 30, 5}}); } TEST_F(DatasetWriterTestFixture, MaxRowsManyWrites) { @@ -196,8 +227,7 @@ TEST_F(DatasetWriterTestFixture, ConcurrentWritesDifferentFiles) { for (int i = 0; i < NBATCHES; i++) { std::string i_str = std::to_string(i); expected_files.push_back(ExpectedFile{"testdir/part" + i_str + "/part-0.arrow", - static_cast(i) * 10, - (static_cast(i + 1) * 10)}); + static_cast(i) * 10, 10}); Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(10), "part" + i_str); AssertFinished(queue_fut); ASSERT_FINISHES_OK(queue_fut); @@ -235,7 +265,8 @@ TEST_F(DatasetWriterTestFixture, MaxOpenFiles) { {"testdir/part0/part-1.arrow", 40, 10}, {"testdir/part1/part-0.arrow", 10, 10}, {"testdir/part1/part-0.arrow", 50, 10}, - {"testdir/part2/part-0.arrow", 30, 10}}); + {"testdir/part2/part-0.arrow", 30, 10}}, + /*verify_content=*/false); } TEST_F(DatasetWriterTestFixture, DeleteExistingData) { From f21d6deb3a65e475fc0fc218eb6f15104337f977 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Mon, 20 Sep 2021 19:09:50 -1000 Subject: [PATCH 18/26] ARROW-13542: Added condition to skip empty batches --- cpp/src/arrow/dataset/dataset_writer.cc | 3 +++ 1 file changed, 3 insertions(+) diff --git a/cpp/src/arrow/dataset/dataset_writer.cc b/cpp/src/arrow/dataset/dataset_writer.cc index d9bc2890a2a..fe823f87b16 100644 --- a/cpp/src/arrow/dataset/dataset_writer.cc +++ b/cpp/src/arrow/dataset/dataset_writer.cc @@ -386,6 +386,9 @@ class DatasetWriter::DatasetWriterImpl : public util::AsyncDestroyable { Future<> WriteRecordBatch(std::shared_ptr batch, const std::string& directory) { RETURN_NOT_OK(CheckError()); + if (batch->num_rows() == 0) { + return Future<>::MakeFinished(); + } if (!directory.empty()) { auto full_path = fs::internal::ConcatAbstractPath(write_options_.base_dir, directory); From 0ff44b786b77f284a26ff027042fe5ec69451b9b Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 28 Sep 2021 01:51:40 -1000 Subject: [PATCH 19/26] ARROW-13542: Addressing comments from PR review --- cpp/src/arrow/dataset/dataset_writer.cc | 13 +++++++++---- cpp/src/arrow/dataset/file_base.cc | 6 +++--- 2 files changed, 12 insertions(+), 7 deletions(-) diff --git a/cpp/src/arrow/dataset/dataset_writer.cc b/cpp/src/arrow/dataset/dataset_writer.cc index fe823f87b16..467757a4233 100644 --- a/cpp/src/arrow/dataset/dataset_writer.cc +++ b/cpp/src/arrow/dataset/dataset_writer.cc @@ -351,6 +351,11 @@ Status ValidateBasenameTemplate(util::string_view basename_template) { if (token_start == util::string_view::npos) { return Status::Invalid("basename_template did not contain '", kIntegerToken, "'"); } + size_t next_token_start = basename_template.find(kIntegerToken, token_start + 1); + if (next_token_start != util::string_view::npos) { + return Status::Invalid("basename_template contained '", kIntegerToken, + "' more than once"); + } return Status::OK(); } @@ -381,7 +386,7 @@ class DatasetWriter::DatasetWriterImpl : public util::AsyncDestroyable { DatasetWriterImpl(FileSystemDatasetWriteOptions write_options, uint64_t max_rows_queued) : write_options_(std::move(write_options)), rows_in_flight_throttle_(max_rows_queued), - open_files_throttle_(write_options.max_open_files) {} + open_files_throttle_(write_options_.max_open_files) {} Future<> WriteRecordBatch(std::shared_ptr batch, const std::string& directory) { @@ -451,14 +456,14 @@ class DatasetWriter::DatasetWriterImpl : public util::AsyncDestroyable { } for (auto& scheduled_write : scheduled_writes) { - // One of the below callbacks could run immediately and set err_ so we check - // it each time through the loop - RETURN_NOT_OK(CheckError()); RETURN_NOT_OK(task_group_.AddTask(scheduled_write.Then( [this](const WriteTask& write) { rows_in_flight_throttle_.Release(write.num_rows); }, [this](const Status& err) { SetError(err); }))); + // The previously added callback could run immediately and set err_ so we check + // it each time through the loop + RETURN_NOT_OK(CheckError()); } if (batch) { return backpressure.Then( diff --git a/cpp/src/arrow/dataset/file_base.cc b/cpp/src/arrow/dataset/file_base.cc index 90fa48a783e..100e668611c 100644 --- a/cpp/src/arrow/dataset/file_base.cc +++ b/cpp/src/arrow/dataset/file_base.cc @@ -109,9 +109,9 @@ Result> FileFormat::MakeFragment( std::move(partition_expression), std::move(physical_schema))); } -// TODO(ARROW-12355[CSV], ARROW-11772[IPC], ARROW-11843[Parquet]) The following -// implementation of ScanBatchesAsync is both ugly and terribly inefficient. Each of the -// formats should provide their own efficient implementation. +// The following implementation of ScanBatchesAsync is both ugly and terribly inefficient. +// Each of the formats should provide their own efficient implementation. However, this +// is a reasonable starting point or implementation for a dummy/mock format. Result FileFormat::ScanBatchesAsync( const std::shared_ptr& scan_options, const std::shared_ptr& file) const { From 98979fe4a0c1588389ea7b850b4585bdda5e4e1f Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 28 Sep 2021 03:05:14 -1000 Subject: [PATCH 20/26] ARROW-13542: Fixing new test that was added with rebase and was using sync scanner - write dataset --- python/pyarrow/tests/test_dataset.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyarrow/tests/test_dataset.py b/python/pyarrow/tests/test_dataset.py index e121c6b9893..c35884221b5 100644 --- a/python/pyarrow/tests/test_dataset.py +++ b/python/pyarrow/tests/test_dataset.py @@ -3361,8 +3361,8 @@ def test_write_dataset_with_scanner(tempdir): dataset = ds.dataset(tempdir, partitioning=["b"]) with tempfile.TemporaryDirectory() as tempdir2: - ds.write_dataset(dataset.scanner(columns=["b", "c"]), tempdir2, - format='parquet', partitioning=["b"]) + ds.write_dataset(dataset.scanner(columns=["b", "c"], use_async=True), + tempdir2, format='parquet', partitioning=["b"]) load_back = ds.dataset(tempdir2, partitioning=["b"]) load_back_table = load_back.to_table() From e06bf5376e345e4f65ea14fd8cc018f7737d722d Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 28 Sep 2021 15:50:32 -1000 Subject: [PATCH 21/26] Apply suggestions from code review Co-authored-by: Joris Van den Bossche Co-authored-by: Neal Richardson --- cpp/src/arrow/dataset/dataset_writer.h | 2 +- r/tests/testthat/test-metadata.R | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/dataset/dataset_writer.h b/cpp/src/arrow/dataset/dataset_writer.h index 21c9fad4491..5d17236537a 100644 --- a/cpp/src/arrow/dataset/dataset_writer.h +++ b/cpp/src/arrow/dataset/dataset_writer.h @@ -33,7 +33,7 @@ constexpr uint64_t kDefaultDatasetWriterMaxRowsQueued = 64 * 1024 * 1024; /// \brief Utility class that manages a set of writers to different paths /// /// Writers may be closed and reopened (and a new file created) based on the dataset -/// write options (for example, min_rows_per_file or max_open_files) +/// write options (for example, max_rows_per_file or max_open_files) /// /// The dataset writer enforces its own back pressure based on the # of rows (as opposed /// to # of batches which is how it is typically enforced elsewhere) and # of files. diff --git a/r/tests/testthat/test-metadata.R b/r/tests/testthat/test-metadata.R index 34c6b536f87..910ccb13773 100644 --- a/r/tests/testthat/test-metadata.R +++ b/r/tests/testthat/test-metadata.R @@ -246,7 +246,7 @@ test_that("metadata of list elements (ARROW-10386)", { "Row-level metadata is not compatible with datasets and will be discarded" ) - # Reset directory as previous write will have created some files and the defualt + # Reset directory as previous write will have created some files and the default # behavior is to error on existing dst_dir <- make_temp_dir() # but we need to write a dataset with row-level metadata to make sure when From 3d354df7f6e41094230caf602271515dad6d7751 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 28 Sep 2021 15:52:00 -1000 Subject: [PATCH 22/26] Update cpp/src/arrow/dataset/dataset_writer.cc Co-authored-by: Joris Van den Bossche --- cpp/src/arrow/dataset/dataset_writer.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/dataset/dataset_writer.cc b/cpp/src/arrow/dataset/dataset_writer.cc index 467757a4233..c2564b7d78b 100644 --- a/cpp/src/arrow/dataset/dataset_writer.cc +++ b/cpp/src/arrow/dataset/dataset_writer.cc @@ -373,7 +373,7 @@ Status EnsureDestinationValid(const FileSystemDatasetWriteOptions& options) { if (maybe_files->size() > 1) { return Status::Invalid( "Could not write to ", options.base_dir, - " as the directory is not empty and existing_data_behavior is kError"); + " as the directory is not empty and existing_data_behavior is to error"); } } return Status::OK(); From 6759e3d79dd2e5c1f830c028bbbeb39a8e82a297 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Tue, 28 Sep 2021 16:46:45 -1000 Subject: [PATCH 23/26] ARROW-13542: Addressing feedback from code review --- cpp/src/arrow/dataset/dataset_writer.cc | 5 +- cpp/src/arrow/dataset/dataset_writer_test.cc | 72 +++++++++++--------- cpp/src/arrow/dataset/file_base.h | 2 + cpp/src/arrow/filesystem/filesystem.cc | 13 +--- cpp/src/arrow/filesystem/filesystem.h | 5 +- python/pyarrow/tests/test_dataset.py | 2 - 6 files changed, 44 insertions(+), 55 deletions(-) diff --git a/cpp/src/arrow/dataset/dataset_writer.cc b/cpp/src/arrow/dataset/dataset_writer.cc index c2564b7d78b..daebb1186d0 100644 --- a/cpp/src/arrow/dataset/dataset_writer.cc +++ b/cpp/src/arrow/dataset/dataset_writer.cc @@ -300,10 +300,7 @@ class DatasetWriterDirectoryQueue : public util::AsyncDestroyable { DeferNotOk(write_options_.filesystem->io_context().executor()->Submit([this] { RETURN_NOT_OK(write_options_.filesystem->CreateDir(directory_)); if (write_options_.existing_data_behavior == kDeleteMatchingPartitions) { - fs::FileSelector selector; - selector.base_dir = directory_; - selector.recursive = true; - return write_options_.filesystem->DeleteFiles(selector); + return write_options_.filesystem->DeleteDirContents(directory_); } return Status::OK(); })); diff --git a/cpp/src/arrow/dataset/dataset_writer_test.cc b/cpp/src/arrow/dataset/dataset_writer_test.cc index c1ade277ecb..d4b3e8edf14 100644 --- a/cpp/src/arrow/dataset/dataset_writer_test.cc +++ b/cpp/src/arrow/dataset/dataset_writer_test.cc @@ -126,25 +126,34 @@ class DatasetWriterTestFixture : public testing::Test { return batch; } - void AssertFiles(const std::vector& expected_files, - bool verify_content = true) { + void AssertFileCreated(const util::optional& maybe_file, + const std::string& expected_filename) { + ASSERT_TRUE(maybe_file.has_value()) + << "The file " << expected_filename << " was not created"; + { + SCOPED_TRACE("pre_finish"); + AssertVisited(pre_finish_visited_, expected_filename); + } + { + SCOPED_TRACE("post_finish"); + AssertVisited(post_finish_visited_, expected_filename); + } + } + + void AssertCreatedData(const std::vector& expected_files) { counter_ = 0; for (const auto& expected_file : expected_files) { util::optional written_file = FindFile(expected_file.filename); - ASSERT_TRUE(written_file.has_value()) - << "The file " << expected_file.filename << " was not created"; - { - SCOPED_TRACE("pre_finish"); - AssertVisited(pre_finish_visited_, expected_file.filename); - } - { - SCOPED_TRACE("post_finish"); - AssertVisited(post_finish_visited_, expected_file.filename); - } - if (verify_content) { - AssertBatchesEqual(*MakeBatch(expected_file.start, expected_file.num_rows), - *ReadAsBatch(written_file->data)); - } + AssertFileCreated(written_file, expected_file.filename); + AssertBatchesEqual(*MakeBatch(expected_file.start, expected_file.num_rows), + *ReadAsBatch(written_file->data)); + } + } + + void AssertFilesCreated(const std::vector& expected_files) { + for (const std::string& expected_file : expected_files) { + util::optional written_file = FindFile(expected_file); + AssertFileCreated(written_file, expected_file); } } @@ -176,7 +185,7 @@ TEST_F(DatasetWriterTestFixture, Basic) { Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), ""); AssertFinished(queue_fut); ASSERT_FINISHES_OK(dataset_writer->Finish()); - AssertFiles({{"testdir/part-0.arrow", 0, 100}}); + AssertCreatedData({{"testdir/part-0.arrow", 0, 100}}); } TEST_F(DatasetWriterTestFixture, MaxRowsOneWrite) { @@ -185,10 +194,10 @@ TEST_F(DatasetWriterTestFixture, MaxRowsOneWrite) { Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(35), ""); AssertFinished(queue_fut); ASSERT_FINISHES_OK(dataset_writer->Finish()); - AssertFiles({{"testdir/part-0.arrow", 0, 10}, - {"testdir/part-1.arrow", 10, 10}, - {"testdir/part-2.arrow", 20, 10}, - {"testdir/part-3.arrow", 30, 5}}); + AssertCreatedData({{"testdir/part-0.arrow", 0, 10}, + {"testdir/part-1.arrow", 10, 10}, + {"testdir/part-2.arrow", 20, 10}, + {"testdir/part-3.arrow", 30, 5}}); } TEST_F(DatasetWriterTestFixture, MaxRowsManyWrites) { @@ -201,7 +210,7 @@ TEST_F(DatasetWriterTestFixture, MaxRowsManyWrites) { ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); ASSERT_FINISHES_OK(dataset_writer->Finish()); - AssertFiles({{"testdir/part-0.arrow", 0, 10}, {"testdir/part-1.arrow", 10, 8}}); + AssertCreatedData({{"testdir/part-0.arrow", 0, 10}, {"testdir/part-1.arrow", 10, 8}}); } TEST_F(DatasetWriterTestFixture, ConcurrentWritesSameFile) { @@ -215,7 +224,7 @@ TEST_F(DatasetWriterTestFixture, ConcurrentWritesSameFile) { ASSERT_OK(gated_fs->WaitForOpenOutputStream(1)); ASSERT_OK(gated_fs->UnlockOpenOutputStream(1)); ASSERT_FINISHES_OK(dataset_writer->Finish()); - AssertFiles({{"testdir/part-0.arrow", 0, 100}}); + AssertCreatedData({{"testdir/part-0.arrow", 0, 100}}); } TEST_F(DatasetWriterTestFixture, ConcurrentWritesDifferentFiles) { @@ -235,7 +244,7 @@ TEST_F(DatasetWriterTestFixture, ConcurrentWritesDifferentFiles) { ASSERT_OK(gated_fs->WaitForOpenOutputStream(NBATCHES)); ASSERT_OK(gated_fs->UnlockOpenOutputStream(NBATCHES)); ASSERT_FINISHES_OK(dataset_writer->Finish()); - AssertFiles(expected_files); + AssertCreatedData(expected_files); } TEST_F(DatasetWriterTestFixture, MaxOpenFiles) { @@ -260,13 +269,8 @@ TEST_F(DatasetWriterTestFixture, MaxOpenFiles) { // write may have already been finished ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(10), "part1")); ASSERT_FINISHES_OK(dataset_writer->Finish()); - AssertFiles({{"testdir/part0/part-0.arrow", 0, 10}, - {"testdir/part0/part-0.arrow", 20, 10}, - {"testdir/part0/part-1.arrow", 40, 10}, - {"testdir/part1/part-0.arrow", 10, 10}, - {"testdir/part1/part-0.arrow", 50, 10}, - {"testdir/part2/part-0.arrow", 30, 10}}, - /*verify_content=*/false); + AssertFilesCreated({"testdir/part0/part-0.arrow", "testdir/part0/part-1.arrow", + "testdir/part1/part-0.arrow", "testdir/part2/part-0.arrow"}); } TEST_F(DatasetWriterTestFixture, DeleteExistingData) { @@ -282,7 +286,7 @@ TEST_F(DatasetWriterTestFixture, DeleteExistingData) { Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), ""); AssertFinished(queue_fut); ASSERT_FINISHES_OK(dataset_writer->Finish()); - AssertFiles({{"testdir/part-0.arrow", 0, 100}}); + AssertCreatedData({{"testdir/part-0.arrow", 0, 100}}); AssertNotFiles({"testdir/part-5.arrow", "testdir/blah.txt"}); } @@ -300,7 +304,7 @@ TEST_F(DatasetWriterTestFixture, PartitionedDeleteExistingData) { Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), "part0"); AssertFinished(queue_fut); ASSERT_FINISHES_OK(dataset_writer->Finish()); - AssertFiles({{"testdir/part0/part-0.arrow", 0, 100}}); + AssertCreatedData({{"testdir/part0/part-0.arrow", 0, 100}}); AssertNotFiles({"testdir/part0/foo.arrow"}); AssertEmptyFiles({"testdir/part1/bar.arrow"}); } @@ -319,7 +323,7 @@ TEST_F(DatasetWriterTestFixture, LeaveExistingData) { Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), ""); AssertFinished(queue_fut); ASSERT_FINISHES_OK(dataset_writer->Finish()); - AssertFiles({{"testdir/part-0.arrow", 0, 100}}); + AssertCreatedData({{"testdir/part-0.arrow", 0, 100}}); AssertEmptyFiles({"testdir/part-5.arrow", "testdir/blah.txt"}); } diff --git a/cpp/src/arrow/dataset/file_base.h b/cpp/src/arrow/dataset/file_base.h index ba03dd0ad27..fc5e17b0c2d 100644 --- a/cpp/src/arrow/dataset/file_base.h +++ b/cpp/src/arrow/dataset/file_base.h @@ -383,6 +383,8 @@ struct ARROW_DS_EXPORT FileSystemDatasetWriteOptions { uint32_t max_open_files = 1024; /// If greater than 0 then this will limit how many rows are placed in any single file. + /// Otherwise there will be no limit and one file will be created in each output + /// directory unless files need to be closed to respect max_open_files uint64_t max_rows_per_file = 0; /// Controls what happens if an output directory already exists. diff --git a/cpp/src/arrow/filesystem/filesystem.cc b/cpp/src/arrow/filesystem/filesystem.cc index a922f0210a3..fbe8b1f1700 100644 --- a/cpp/src/arrow/filesystem/filesystem.cc +++ b/cpp/src/arrow/filesystem/filesystem.cc @@ -15,12 +15,12 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/filesystem/filesystem.h" - #include #include #include "arrow/util/config.h" + +#include "arrow/filesystem/filesystem.h" #ifdef ARROW_HDFS #include "arrow/filesystem/hdfs.h" #endif @@ -176,15 +176,6 @@ Status FileSystem::DeleteFiles(const std::vector& paths) { return st; } -Status FileSystem::DeleteFiles(const FileSelector& select) { - Status st = Status::OK(); - ARROW_ASSIGN_OR_RAISE(auto file_infos, GetFileInfo(select)); - for (const auto& file_info : file_infos) { - st &= DeleteFile(file_info.path()); - } - return st; -} - namespace { Status ValidateInputFileInfo(const FileInfo& info) { diff --git a/cpp/src/arrow/filesystem/filesystem.h b/cpp/src/arrow/filesystem/filesystem.h index 0326ed8f95e..6a36d51e929 100644 --- a/cpp/src/arrow/filesystem/filesystem.h +++ b/cpp/src/arrow/filesystem/filesystem.h @@ -202,7 +202,7 @@ class ARROW_EXPORT FileSystem : public std::enable_shared_from_this /// the returned future to complete before calling the generator again. virtual FileInfoGenerator GetFileInfoGenerator(const FileSelector& select); - /// Create a directory and parent directories. + /// Create a directory and subdirectories. /// /// This function succeeds if the directory already exists. virtual Status CreateDir(const std::string& path, bool recursive = true) = 0; @@ -230,9 +230,6 @@ class ARROW_EXPORT FileSystem : public std::enable_shared_from_this /// The default implementation issues individual delete operations in sequence. virtual Status DeleteFiles(const std::vector& paths); - /// Selector based override of DeleteFiles - virtual Status DeleteFiles(const FileSelector& selector); - /// Move / rename a file or directory. /// /// If the destination exists: diff --git a/python/pyarrow/tests/test_dataset.py b/python/pyarrow/tests/test_dataset.py index c35884221b5..6d84c8cce21 100644 --- a/python/pyarrow/tests/test_dataset.py +++ b/python/pyarrow/tests/test_dataset.py @@ -3714,8 +3714,6 @@ def file_visitor(written_file): use_threads=True, file_visitor=file_visitor ) - # Since it is a multi-threaded write there is no way to know which - # directory gets part-0 and which gets part-1 expected_paths = { root_path / 'part=a' / 'part-0.parquet', root_path / 'part=b' / 'part-0.parquet' From 2ac84d472f293a715dbb9a26111c5e7a5cfd5e4f Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Wed, 29 Sep 2021 09:08:30 -1000 Subject: [PATCH 24/26] ARROW-13542: Addressing feedback from PR review --- cpp/src/arrow/dataset/dataset_writer.cc | 2 + cpp/src/arrow/dataset/dataset_writer.h | 8 ++- cpp/src/arrow/dataset/dataset_writer_test.cc | 71 +++++++++++--------- cpp/src/arrow/dataset/file_base.cc | 5 +- cpp/src/arrow/filesystem/s3_test_util.h | 5 +- cpp/src/arrow/filesystem/test_util.cc | 11 ++- cpp/src/arrow/filesystem/test_util.h | 10 +++ cpp/src/arrow/util/CMakeLists.txt | 2 +- cpp/src/arrow/util/counting_semaphore.h | 4 +- r/R/dataset-write.R | 2 +- 10 files changed, 69 insertions(+), 51 deletions(-) diff --git a/cpp/src/arrow/dataset/dataset_writer.cc b/cpp/src/arrow/dataset/dataset_writer.cc index daebb1186d0..6233b4bf4af 100644 --- a/cpp/src/arrow/dataset/dataset_writer.cc +++ b/cpp/src/arrow/dataset/dataset_writer.cc @@ -31,6 +31,7 @@ namespace arrow { namespace dataset { +namespace internal { namespace { @@ -522,5 +523,6 @@ Future<> DatasetWriter::Finish() { return finished; } +} // namespace internal } // namespace dataset } // namespace arrow diff --git a/cpp/src/arrow/dataset/dataset_writer.h b/cpp/src/arrow/dataset/dataset_writer.h index 5d17236537a..b014f963554 100644 --- a/cpp/src/arrow/dataset/dataset_writer.h +++ b/cpp/src/arrow/dataset/dataset_writer.h @@ -27,6 +27,7 @@ namespace arrow { namespace dataset { +namespace internal { constexpr uint64_t kDefaultDatasetWriterMaxRowsQueued = 64 * 1024 * 1024; @@ -39,7 +40,7 @@ constexpr uint64_t kDefaultDatasetWriterMaxRowsQueued = 64 * 1024 * 1024; /// to # of batches which is how it is typically enforced elsewhere) and # of files. class ARROW_DS_EXPORT DatasetWriter { public: - /// \brief Creates a dataset writer + /// \brief Create a dataset writer /// /// Will fail if basename_template is invalid or if there is existing data and /// existing_data_behavior is kError @@ -53,7 +54,7 @@ class ARROW_DS_EXPORT DatasetWriter { ~DatasetWriter(); - /// \brief Writes a batch to the dataset + /// \brief Write a batch to the dataset /// \param[in] batch The batch to write /// \param[in] directory The directory to write to /// @@ -67,7 +68,7 @@ class ARROW_DS_EXPORT DatasetWriter { /// to be written. If the returned future is unfinished then this indicates the dataset /// writer's queue is full and the data provider should pause. /// - /// This method is NOT async reentrant. The returned future will only be incomplete + /// This method is NOT async reentrant. The returned future will only be unfinished /// if back pressure needs to be applied. Async reentrancy is not necessary for /// concurrent writes to happen. Calling this method again before the previous future /// completes will not just violate max_rows_queued but likely lead to race conditions. @@ -91,5 +92,6 @@ class ARROW_DS_EXPORT DatasetWriter { std::unique_ptr> impl_; }; +} // namespace internal } // namespace dataset } // namespace arrow diff --git a/cpp/src/arrow/dataset/dataset_writer_test.cc b/cpp/src/arrow/dataset/dataset_writer_test.cc index d4b3e8edf14..e3fac05ccd3 100644 --- a/cpp/src/arrow/dataset/dataset_writer_test.cc +++ b/cpp/src/arrow/dataset/dataset_writer_test.cc @@ -34,18 +34,19 @@ namespace arrow { namespace dataset { +namespace internal { using arrow::fs::internal::MockFileInfo; using arrow::fs::internal::MockFileSystem; -struct ExpectedFile { - std::string filename; - uint64_t start; - uint64_t num_rows; -}; - class DatasetWriterTestFixture : public testing::Test { protected: + struct ExpectedFile { + std::string filename; + uint64_t start; + uint64_t num_rows; + }; + void SetUp() override { fs::TimePoint mock_now = std::chrono::system_clock::now(); ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, @@ -53,7 +54,7 @@ class DatasetWriterTestFixture : public testing::Test { filesystem_ = std::dynamic_pointer_cast(fs); schema_ = schema({field("int64", int64())}); write_options_.filesystem = filesystem_; - write_options_.basename_template = "part-{i}.arrow"; + write_options_.basename_template = "chunk-{i}.arrow"; write_options_.base_dir = "testdir"; write_options_.writer_pre_finish = [this](FileWriter* writer) { pre_finish_visited_.push_back(writer->destination().path); @@ -102,8 +103,7 @@ class DatasetWriterTestFixture : public testing::Test { void AssertVisited(const std::vector& actual_paths, const std::string& expected_path) { - std::vector::const_iterator found = - std::find(actual_paths.begin(), actual_paths.end(), expected_path); + const auto found = std::find(actual_paths.begin(), actual_paths.end(), expected_path); ASSERT_NE(found, actual_paths.end()) << "The file " << expected_path << " was not in the list of files visited"; } @@ -185,7 +185,7 @@ TEST_F(DatasetWriterTestFixture, Basic) { Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), ""); AssertFinished(queue_fut); ASSERT_FINISHES_OK(dataset_writer->Finish()); - AssertCreatedData({{"testdir/part-0.arrow", 0, 100}}); + AssertCreatedData({{"testdir/chunk-0.arrow", 0, 100}}); } TEST_F(DatasetWriterTestFixture, MaxRowsOneWrite) { @@ -194,10 +194,10 @@ TEST_F(DatasetWriterTestFixture, MaxRowsOneWrite) { Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(35), ""); AssertFinished(queue_fut); ASSERT_FINISHES_OK(dataset_writer->Finish()); - AssertCreatedData({{"testdir/part-0.arrow", 0, 10}, - {"testdir/part-1.arrow", 10, 10}, - {"testdir/part-2.arrow", 20, 10}, - {"testdir/part-3.arrow", 30, 5}}); + AssertCreatedData({{"testdir/chunk-0.arrow", 0, 10}, + {"testdir/chunk-1.arrow", 10, 10}, + {"testdir/chunk-2.arrow", 20, 10}, + {"testdir/chunk-3.arrow", 30, 5}}); } TEST_F(DatasetWriterTestFixture, MaxRowsManyWrites) { @@ -210,10 +210,12 @@ TEST_F(DatasetWriterTestFixture, MaxRowsManyWrites) { ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(3), "")); ASSERT_FINISHES_OK(dataset_writer->Finish()); - AssertCreatedData({{"testdir/part-0.arrow", 0, 10}, {"testdir/part-1.arrow", 10, 8}}); + AssertCreatedData({{"testdir/chunk-0.arrow", 0, 10}, {"testdir/chunk-1.arrow", 10, 8}}); } TEST_F(DatasetWriterTestFixture, ConcurrentWritesSameFile) { + // Use a gated filesystem to queue up many writes behind a file open to make sure the + // file isn't opened multiple times. auto gated_fs = UseGatedFs(); EXPECT_OK_AND_ASSIGN(auto dataset_writer, DatasetWriter::Make(write_options_)); for (int i = 0; i < 10; i++) { @@ -224,7 +226,7 @@ TEST_F(DatasetWriterTestFixture, ConcurrentWritesSameFile) { ASSERT_OK(gated_fs->WaitForOpenOutputStream(1)); ASSERT_OK(gated_fs->UnlockOpenOutputStream(1)); ASSERT_FINISHES_OK(dataset_writer->Finish()); - AssertCreatedData({{"testdir/part-0.arrow", 0, 100}}); + AssertCreatedData({{"testdir/chunk-0.arrow", 0, 100}}); } TEST_F(DatasetWriterTestFixture, ConcurrentWritesDifferentFiles) { @@ -235,7 +237,7 @@ TEST_F(DatasetWriterTestFixture, ConcurrentWritesDifferentFiles) { EXPECT_OK_AND_ASSIGN(auto dataset_writer, DatasetWriter::Make(write_options_)); for (int i = 0; i < NBATCHES; i++) { std::string i_str = std::to_string(i); - expected_files.push_back(ExpectedFile{"testdir/part" + i_str + "/part-0.arrow", + expected_files.push_back(ExpectedFile{"testdir/part" + i_str + "/chunk-0.arrow", static_cast(i) * 10, 10}); Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(10), "part" + i_str); AssertFinished(queue_fut); @@ -269,16 +271,17 @@ TEST_F(DatasetWriterTestFixture, MaxOpenFiles) { // write may have already been finished ASSERT_FINISHES_OK(dataset_writer->WriteRecordBatch(MakeBatch(10), "part1")); ASSERT_FINISHES_OK(dataset_writer->Finish()); - AssertFilesCreated({"testdir/part0/part-0.arrow", "testdir/part0/part-1.arrow", - "testdir/part1/part-0.arrow", "testdir/part2/part-0.arrow"}); + AssertFilesCreated({"testdir/part0/chunk-0.arrow", "testdir/part0/chunk-1.arrow", + "testdir/part1/chunk-0.arrow", "testdir/part2/chunk-0.arrow"}); } TEST_F(DatasetWriterTestFixture, DeleteExistingData) { fs::TimePoint mock_now = std::chrono::system_clock::now(); - ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, - MockFileSystem::Make(mock_now, {::arrow::fs::Dir("testdir"), - fs::File("testdir/part-5.arrow"), - fs::File("testdir/blah.txt")})); + ASSERT_OK_AND_ASSIGN( + std::shared_ptr fs, + MockFileSystem::Make( + mock_now, {::arrow::fs::Dir("testdir"), fs::File("testdir/subdir/foo.txt"), + fs::File("testdir/chunk-5.arrow"), fs::File("testdir/blah.txt")})); filesystem_ = std::dynamic_pointer_cast(fs); write_options_.filesystem = filesystem_; write_options_.existing_data_behavior = kDeleteMatchingPartitions; @@ -286,8 +289,8 @@ TEST_F(DatasetWriterTestFixture, DeleteExistingData) { Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), ""); AssertFinished(queue_fut); ASSERT_FINISHES_OK(dataset_writer->Finish()); - AssertCreatedData({{"testdir/part-0.arrow", 0, 100}}); - AssertNotFiles({"testdir/part-5.arrow", "testdir/blah.txt"}); + AssertCreatedData({{"testdir/chunk-0.arrow", 0, 100}}); + AssertNotFiles({"testdir/chunk-5.arrow", "testdir/blah.txt", "testdir/subdir/foo.txt"}); } TEST_F(DatasetWriterTestFixture, PartitionedDeleteExistingData) { @@ -304,7 +307,7 @@ TEST_F(DatasetWriterTestFixture, PartitionedDeleteExistingData) { Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), "part0"); AssertFinished(queue_fut); ASSERT_FINISHES_OK(dataset_writer->Finish()); - AssertCreatedData({{"testdir/part0/part-0.arrow", 0, 100}}); + AssertCreatedData({{"testdir/part0/chunk-0.arrow", 0, 100}}); AssertNotFiles({"testdir/part0/foo.arrow"}); AssertEmptyFiles({"testdir/part1/bar.arrow"}); } @@ -314,8 +317,8 @@ TEST_F(DatasetWriterTestFixture, LeaveExistingData) { ASSERT_OK_AND_ASSIGN( std::shared_ptr fs, MockFileSystem::Make( - mock_now, {::arrow::fs::Dir("testdir"), fs::File("testdir/part-0.arrow"), - fs::File("testdir/part-5.arrow"), fs::File("testdir/blah.txt")})); + mock_now, {::arrow::fs::Dir("testdir"), fs::File("testdir/chunk-0.arrow"), + fs::File("testdir/chunk-5.arrow"), fs::File("testdir/blah.txt")})); filesystem_ = std::dynamic_pointer_cast(fs); write_options_.filesystem = filesystem_; write_options_.existing_data_behavior = kOverwriteOrIgnore; @@ -323,8 +326,8 @@ TEST_F(DatasetWriterTestFixture, LeaveExistingData) { Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), ""); AssertFinished(queue_fut); ASSERT_FINISHES_OK(dataset_writer->Finish()); - AssertCreatedData({{"testdir/part-0.arrow", 0, 100}}); - AssertEmptyFiles({"testdir/part-5.arrow", "testdir/blah.txt"}); + AssertCreatedData({{"testdir/chunk-0.arrow", 0, 100}}); + AssertEmptyFiles({"testdir/chunk-5.arrow", "testdir/blah.txt"}); } TEST_F(DatasetWriterTestFixture, ErrOnExistingData) { @@ -332,13 +335,15 @@ TEST_F(DatasetWriterTestFixture, ErrOnExistingData) { ASSERT_OK_AND_ASSIGN( std::shared_ptr fs, MockFileSystem::Make( - mock_now, {::arrow::fs::Dir("testdir"), fs::File("testdir/part-0.arrow"), - fs::File("testdir/part-5.arrow"), fs::File("testdir/blah.txt")})); + mock_now, {::arrow::fs::Dir("testdir"), fs::File("testdir/chunk-0.arrow"), + fs::File("testdir/chunk-5.arrow"), fs::File("testdir/blah.txt")})); filesystem_ = std::dynamic_pointer_cast(fs); write_options_.filesystem = filesystem_; ASSERT_RAISES(Invalid, DatasetWriter::Make(write_options_)); - AssertEmptyFiles({"testdir/part-0.arrow", "testdir/part-5.arrow", "testdir/blah.txt"}); + AssertEmptyFiles( + {"testdir/chunk-0.arrow", "testdir/chunk-5.arrow", "testdir/blah.txt"}); } +} // namespace internal } // namespace dataset } // namespace arrow diff --git a/cpp/src/arrow/dataset/file_base.cc b/cpp/src/arrow/dataset/file_base.cc index 100e668611c..ec65bf12e23 100644 --- a/cpp/src/arrow/dataset/file_base.cc +++ b/cpp/src/arrow/dataset/file_base.cc @@ -322,7 +322,7 @@ Status FileWriter::Finish() { namespace { -Future<> WriteNextBatch(DatasetWriter* dataset_writer, TaggedRecordBatch batch, +Future<> WriteNextBatch(internal::DatasetWriter* dataset_writer, TaggedRecordBatch batch, const FileSystemDatasetWriteOptions& write_options) { ARROW_ASSIGN_OR_RAISE(auto groups, write_options.partitioning->Partition(batch.record_batch)); @@ -365,7 +365,8 @@ Future<> WriteNextBatch(DatasetWriter* dataset_writer, TaggedRecordBatch batch, Status FileSystemDataset::Write(const FileSystemDatasetWriteOptions& write_options, std::shared_ptr scanner) { ARROW_ASSIGN_OR_RAISE(auto batch_gen, scanner->ScanBatchesAsync()); - ARROW_ASSIGN_OR_RAISE(auto dataset_writer, DatasetWriter::Make(write_options)); + ARROW_ASSIGN_OR_RAISE(auto dataset_writer, + internal::DatasetWriter::Make(write_options)); AsyncGenerator> queued_batch_gen = [batch_gen, &dataset_writer, &write_options]() -> Future> { diff --git a/cpp/src/arrow/filesystem/s3_test_util.h b/cpp/src/arrow/filesystem/s3_test_util.h index 2bb614b9b80..432ff1d22ff 100644 --- a/cpp/src/arrow/filesystem/s3_test_util.h +++ b/cpp/src/arrow/filesystem/s3_test_util.h @@ -25,11 +25,10 @@ // We need BOOST_USE_WINDOWS_H definition with MinGW when we use // boost/process.hpp. See ARROW_BOOST_PROCESS_COMPILE_DEFINITIONS in // cpp/cmake_modules/BuildUtils.cmake for details. -#include - +#include #include -#include +#include #include "arrow/filesystem/s3fs.h" #include "arrow/status.h" diff --git a/cpp/src/arrow/filesystem/test_util.cc b/cpp/src/arrow/filesystem/test_util.cc index da17447a4b2..14c9c7ac11f 100644 --- a/cpp/src/arrow/filesystem/test_util.cc +++ b/cpp/src/arrow/filesystem/test_util.cc @@ -15,17 +15,18 @@ // specific language governing permissions and limitations // under the License. +#include "arrow/filesystem/test_util.h" + +#include +#include + #include #include #include #include #include -#include -#include - #include "arrow/filesystem/mockfs.h" -#include "arrow/filesystem/test_util.h" #include "arrow/io/interfaces.h" #include "arrow/status.h" #include "arrow/testing/future_util.h" @@ -176,12 +177,10 @@ Result> GatedMockFilesystem::OpenOutputStream( return MockFileSystem::OpenOutputStream(path, metadata); } -// Wait until at least num_waiters are waiting on OpenOutputStream Status GatedMockFilesystem::WaitForOpenOutputStream(uint32_t num_waiters) { return open_output_sem_.WaitForWaiters(num_waiters); } -// Unlocks `num_waiters` individual calls to OpenOutputStream Status GatedMockFilesystem::UnlockOpenOutputStream(uint32_t num_waiters) { return open_output_sem_.Release(num_waiters); } diff --git a/cpp/src/arrow/filesystem/test_util.h b/cpp/src/arrow/filesystem/test_util.h index 59852bbe39b..8d80b10bada 100644 --- a/cpp/src/arrow/filesystem/test_util.h +++ b/cpp/src/arrow/filesystem/test_util.h @@ -40,6 +40,14 @@ static inline FileInfo Dir(std::string path) { return FileInfo(std::move(path), FileType::Directory); } +// A subclass of MockFileSystem that blocks operations until an unlock method is +// called. +// +// This is intended for testing fine-grained ordering of filesystem operations. +// +// N.B. Only OpenOutputStream supports gating at the moment but this is simply because +// it is all that has been needed so far. Feel free to add support for more methods +// as required. class ARROW_TESTING_EXPORT GatedMockFilesystem : public internal::MockFileSystem { public: GatedMockFilesystem(TimePoint current_time, @@ -50,7 +58,9 @@ class ARROW_TESTING_EXPORT GatedMockFilesystem : public internal::MockFileSystem const std::string& path, const std::shared_ptr& metadata = {}) override; + // Wait until at least num_waiters are waiting on OpenOutputStream Status WaitForOpenOutputStream(uint32_t num_waiters); + // Unlock `num_waiters` individual calls to OpenOutputStream Status UnlockOpenOutputStream(uint32_t num_waiters); private: diff --git a/cpp/src/arrow/util/CMakeLists.txt b/cpp/src/arrow/util/CMakeLists.txt index bc44ef729d6..6d36fde930b 100644 --- a/cpp/src/arrow/util/CMakeLists.txt +++ b/cpp/src/arrow/util/CMakeLists.txt @@ -48,7 +48,6 @@ add_arrow_test(utility-test cache_test.cc checked_cast_test.cc compression_test.cc - counting_semaphore_test.cc decimal_test.cc formatting_util_test.cc key_value_metadata_test.cc @@ -76,6 +75,7 @@ add_arrow_test(utility-test add_arrow_test(threading-utility-test SOURCES cancel_test.cc + counting_semaphore_test.cc future_test.cc task_group_test.cc thread_pool_test.cc) diff --git a/cpp/src/arrow/util/counting_semaphore.h b/cpp/src/arrow/util/counting_semaphore.h index 317636190ce..a3c13cc3bea 100644 --- a/cpp/src/arrow/util/counting_semaphore.h +++ b/cpp/src/arrow/util/counting_semaphore.h @@ -39,11 +39,11 @@ class ARROW_EXPORT CountingSemaphore { Status Acquire(uint32_t num_permits); /// \brief Make num_permits permits available Status Release(uint32_t num_permits); - /// \brief Waits until num_waiters are waiting on permits + /// \brief Wait until num_waiters are waiting on permits /// /// This method is non-standard but useful in unit tests to ensure sequencing Status WaitForWaiters(uint32_t num_waiters); - /// \brief Immediately times out any waiters + /// \brief Immediately time out any waiters /// /// This method will return Status::OK only if there were no waiters to time out. /// Once closed any operation on this instance will return an invalid status. diff --git a/r/R/dataset-write.R b/r/R/dataset-write.R index 1253445de19..5933fd99ab5 100644 --- a/r/R/dataset-write.R +++ b/r/R/dataset-write.R @@ -111,7 +111,7 @@ write_dataset <- function(dataset, dataset <- dplyr::ungroup(dataset) } - scanner <- Scanner$create(dataset, use_async=TRUE) + scanner <- Scanner$create(dataset, use_async = TRUE) if (!inherits(partitioning, "Partitioning")) { partition_schema <- scanner$schema[partitioning] if (isTRUE(hive_style)) { From 8f0d17b5c73e28a8169fa9acfd8a6034c8df8e94 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Wed, 29 Sep 2021 15:38:25 -1000 Subject: [PATCH 25/26] ARROW-13542: Undoing changes not related to the issue --- cpp/src/arrow/filesystem/test_util.cc | 9 ++++----- cpp/src/arrow/testing/gtest_util.h | 2 +- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/cpp/src/arrow/filesystem/test_util.cc b/cpp/src/arrow/filesystem/test_util.cc index 14c9c7ac11f..a71ce11d699 100644 --- a/cpp/src/arrow/filesystem/test_util.cc +++ b/cpp/src/arrow/filesystem/test_util.cc @@ -15,17 +15,16 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/filesystem/test_util.h" - -#include -#include - #include #include #include #include #include +#include +#include + +#include "arrow/filesystem/test_util.h" #include "arrow/filesystem/mockfs.h" #include "arrow/io/interfaces.h" #include "arrow/status.h" diff --git a/cpp/src/arrow/testing/gtest_util.h b/cpp/src/arrow/testing/gtest_util.h index e1baec50ae2..da145bdfa01 100644 --- a/cpp/src/arrow/testing/gtest_util.h +++ b/cpp/src/arrow/testing/gtest_util.h @@ -94,7 +94,7 @@ #define ARROW_EXPECT_OK(expr) \ do { \ - const auto& _res = (expr); \ + auto _res = (expr); \ ::arrow::Status _st = ::arrow::internal::GenericToStatus(_res); \ EXPECT_TRUE(_st.ok()) << "'" ARROW_STRINGIFY(expr) "' failed with " \ << _st.ToString(); \ From f352036adace5cda4546ac20f12340d30a33e209 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Wed, 29 Sep 2021 16:45:20 -1000 Subject: [PATCH 26/26] ARROW-13542: Fixing up import order for lint --- cpp/src/arrow/filesystem/test_util.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/filesystem/test_util.cc b/cpp/src/arrow/filesystem/test_util.cc index a71ce11d699..0e2833781ec 100644 --- a/cpp/src/arrow/filesystem/test_util.cc +++ b/cpp/src/arrow/filesystem/test_util.cc @@ -24,8 +24,8 @@ #include #include -#include "arrow/filesystem/test_util.h" #include "arrow/filesystem/mockfs.h" +#include "arrow/filesystem/test_util.h" #include "arrow/io/interfaces.h" #include "arrow/status.h" #include "arrow/testing/future_util.h"