diff --git a/ci/scripts/r_docker_configure.sh b/ci/scripts/r_docker_configure.sh index 26c19ca750403..853f03267bd45 100755 --- a/ci/scripts/r_docker_configure.sh +++ b/ci/scripts/r_docker_configure.sh @@ -76,7 +76,7 @@ if [[ -n "$DEVTOOLSET_VERSION" ]]; then # Only add make var if not set if ! grep -Fq "CXX17=" ~/.R/Makevars &> /dev/null; then mkdir -p ~/.R - echo "CXX17=g++ -std=g++17 -g -O2 -fpic" >> ~/.R/Makevars + echo "CXX17=g++ -std=gnu++17 -g -O2 -fpic" >> ~/.R/Makevars fi fi diff --git a/cpp/src/arrow/compute/cast.cc b/cpp/src/arrow/compute/cast.cc index 710e9c831b0bf..99e8b89f1ca13 100644 --- a/cpp/src/arrow/compute/cast.cc +++ b/cpp/src/arrow/compute/cast.cc @@ -98,8 +98,25 @@ class CastMetaFunction : public MetaFunction { // args[0].type() could be a nullptr so check for that before // we do anything with it. if (args[0].type() && args[0].type()->Equals(*cast_options->to_type)) { - return args[0]; + // Nested types might differ in field names but still be considered equal, + // so we can only return non-nested types as-is. + if (!is_nested(args[0].type()->id())) { + return args[0]; + } else if (args[0].is_array()) { + // TODO(ARROW-14999): if types are equal except for field names of list + // types, we can also use this code path. + ARROW_ASSIGN_OR_RAISE(std::shared_ptr array, + ::arrow::internal::GetArrayView( + args[0].array(), cast_options->to_type.owned_type)); + return Datum(array); + } else if (args[0].is_chunked_array()) { + ARROW_ASSIGN_OR_RAISE( + std::shared_ptr array, + args[0].chunked_array()->View(cast_options->to_type.owned_type)); + return Datum(array); + } } + Result> result = GetCastFunction(*cast_options->to_type); if (!result.ok()) { diff --git a/cpp/src/arrow/compute/exec/expression.cc b/cpp/src/arrow/compute/exec/expression.cc index e99f69fbca7fe..c6b2ddb3b1f9b 100644 --- a/cpp/src/arrow/compute/exec/expression.cc +++ b/cpp/src/arrow/compute/exec/expression.cc @@ -196,11 +196,11 @@ std::string Expression::ToString() const { if (call->options) { out += call->options->ToString(); - out.resize(out.size() + 1); - } else { - out.resize(out.size() - 1); + } else if (call->arguments.size()) { + out.resize(out.size() - 2); } - out.back() = ')'; + + out += ')'; return out; } diff --git a/cpp/src/arrow/compute/exec/expression_test.cc b/cpp/src/arrow/compute/exec/expression_test.cc index 25b38cc80ff70..819aed2023735 100644 --- a/cpp/src/arrow/compute/exec/expression_test.cc +++ b/cpp/src/arrow/compute/exec/expression_test.cc @@ -284,8 +284,7 @@ TEST(Expression, ToString) { "allow_time_overflow=false, allow_decimal_truncate=false, " "allow_float_truncate=false, allow_invalid_utf8=false})"); - // NB: corrupted for nullary functions but we don't have any of those - EXPECT_EQ(call("widgetify", {}).ToString(), "widgetif)"); + EXPECT_EQ(call("widgetify", {}).ToString(), "widgetify()"); EXPECT_EQ( call("widgetify", {literal(1)}, std::make_shared()).ToString(), "widgetify(1, widgetify)"); @@ -312,6 +311,11 @@ TEST(Expression, ToString) { }) .ToString(), "{a=a, renamed_a=a, three=3, b=" + in_12.ToString() + "}"); + + EXPECT_EQ(call("round", {literal(3.14)}, compute::RoundOptions()).ToString(), + "round(3.14, {ndigits=0, round_mode=HALF_TO_EVEN})"); + EXPECT_EQ(call("random", {}, compute::RandomOptions()).ToString(), + "random({initializer=SystemRandom, seed=0})"); } TEST(Expression, Equality) { diff --git a/cpp/src/arrow/compute/kernels/scalar_cast_nested.cc b/cpp/src/arrow/compute/kernels/scalar_cast_nested.cc index d8094a9913b49..5b764efe263b2 100644 --- a/cpp/src/arrow/compute/kernels/scalar_cast_nested.cc +++ b/cpp/src/arrow/compute/kernels/scalar_cast_nested.cc @@ -40,25 +40,22 @@ namespace { // (Large)List -> (Large)List -template -typename std::enable_if::type -CastListOffsets(KernelContext* ctx, const ArraySpan& in_array, ArrayData* out_array) { - return Status::OK(); -} - // TODO(wesm): memory could be preallocated here and it would make // things simpler template -typename std::enable_if::type -CastListOffsets(KernelContext* ctx, const ArraySpan& in_array, ArrayData* out_array) { +Status CastListOffsets(KernelContext* ctx, const ArraySpan& in_array, + ArrayData* out_array) { using src_offset_type = typename SrcType::offset_type; using dest_offset_type = typename DestType::offset_type; - ARROW_ASSIGN_OR_RAISE(out_array->buffers[1], - ctx->Allocate(sizeof(dest_offset_type) * (in_array.length + 1))); - ::arrow::internal::CastInts(in_array.GetValues(1), - out_array->GetMutableValues(1), - in_array.length + 1); + if constexpr (!std::is_same::value) { + ARROW_ASSIGN_OR_RAISE(out_array->buffers[1], ctx->Allocate(sizeof(dest_offset_type) * + (in_array.length + 1))); + ::arrow::internal::CastInts(in_array.GetValues(1), + out_array->GetMutableValues(1), + in_array.length + 1); + } + return Status::OK(); } @@ -70,25 +67,10 @@ struct CastList { static constexpr bool is_upcast = sizeof(src_offset_type) < sizeof(dest_offset_type); static constexpr bool is_downcast = sizeof(src_offset_type) > sizeof(dest_offset_type); - static Status Exec(KernelContext* ctx, const ExecSpan& batch, ExecResult* out) { - const CastOptions& options = CastState::Get(ctx); - - auto child_type = checked_cast(*out->type()).value_type(); - - const ArraySpan& in_array = batch[0].array; + static Status HandleOffsets(KernelContext* ctx, const ArraySpan& in_array, + ArrayData* out_array, std::shared_ptr* values) { auto offsets = in_array.GetValues(1); - ArrayData* out_array = out->array_data().get(); - out_array->buffers[0] = in_array.GetBuffer(0); - out_array->buffers[1] = in_array.GetBuffer(1); - - // Shift bitmap in case the source offset is non-zero - if (in_array.offset != 0 && in_array.buffers[0].data != nullptr) { - ARROW_ASSIGN_OR_RAISE(out_array->buffers[0], - CopyBitmap(ctx->memory_pool(), in_array.buffers[0].data, - in_array.offset, in_array.length)); - } - // Handle list offsets // Several cases can arise: // - the source offset is non-zero, in which case we slice the underlying values @@ -103,8 +85,6 @@ struct CastList { } } - std::shared_ptr values = in_array.child_data[0].ToArrayData(); - if (in_array.offset != 0) { ARROW_ASSIGN_OR_RAISE( out_array->buffers[1], @@ -115,11 +95,36 @@ struct CastList { shifted_offsets[i] = static_cast(offsets[i] - offsets[0]); } - values = values->Slice(offsets[0], offsets[in_array.length]); + *values = (*values)->Slice(offsets[0], offsets[in_array.length]); } else { RETURN_NOT_OK((CastListOffsets(ctx, in_array, out_array))); } + return Status::OK(); + } + + static Status Exec(KernelContext* ctx, const ExecSpan& batch, ExecResult* out) { + const CastOptions& options = CastState::Get(ctx); + + auto child_type = checked_cast(*out->type()).value_type(); + + const ArraySpan& in_array = batch[0].array; + + ArrayData* out_array = out->array_data().get(); + out_array->buffers[0] = in_array.GetBuffer(0); + out_array->buffers[1] = in_array.GetBuffer(1); + + std::shared_ptr values = in_array.child_data[0].ToArrayData(); + + // Shift bitmap in case the source offset is non-zero + if (in_array.offset != 0 && in_array.buffers[0].data != nullptr) { + ARROW_ASSIGN_OR_RAISE(out_array->buffers[0], + CopyBitmap(ctx->memory_pool(), in_array.buffers[0].data, + in_array.offset, in_array.length)); + } + + RETURN_NOT_OK(HandleOffsets(ctx, in_array, out_array, &values)); + // Handle values ARROW_ASSIGN_OR_RAISE(Datum cast_values, Cast(values, child_type, options, ctx->exec_context())); @@ -237,6 +242,74 @@ void AddTypeToTypeCast(CastFunction* func) { DCHECK_OK(func->AddKernel(StructType::type_id, std::move(kernel))); } +template +struct CastMap { + using CastListImpl = CastList; + + static Status Exec(KernelContext* ctx, const ExecSpan& batch, ExecResult* out) { + const CastOptions& options = CastState::Get(ctx); + + std::shared_ptr entry_type = + checked_cast(*out->type()).value_type(); + // Assert is struct with two fields + if (!(entry_type->id() == Type::STRUCT && entry_type->num_fields() == 2)) { + return Status::TypeError( + "Map type must be cast to a list with exactly two fields."); + } + std::shared_ptr key_type = entry_type->field(0)->type(); + std::shared_ptr value_type = entry_type->field(1)->type(); + + const ArraySpan& in_array = batch[0].array; + + ArrayData* out_array = out->array_data().get(); + out_array->buffers[0] = in_array.GetBuffer(0); + out_array->buffers[1] = in_array.GetBuffer(1); + + std::shared_ptr entries = in_array.child_data[0].ToArrayData(); + + // Shift bitmap in case the source offset is non-zero + if (in_array.offset != 0 && in_array.buffers[0].data != nullptr) { + ARROW_ASSIGN_OR_RAISE(out_array->buffers[0], + CopyBitmap(ctx->memory_pool(), in_array.buffers[0].data, + in_array.offset, in_array.length)); + } + + RETURN_NOT_OK(CastListImpl::HandleOffsets(ctx, in_array, out_array, &entries)); + + // Handle keys + const std::shared_ptr& keys = + entries->child_data[0]->Slice(entries->offset, entries->length); + ARROW_ASSIGN_OR_RAISE(Datum cast_keys, + Cast(keys, key_type, options, ctx->exec_context())); + DCHECK(cast_keys.is_array()); + + // Handle values + const std::shared_ptr& values = + entries->child_data[1]->Slice(entries->offset, entries->length); + ARROW_ASSIGN_OR_RAISE(Datum cast_values, + Cast(values, value_type, options, ctx->exec_context())); + DCHECK(cast_values.is_array()); + + // Create struct array + std::shared_ptr struct_array = + ArrayData::Make(entry_type, /*length=*/entries->length, {nullptr}, + {cast_keys.array(), cast_values.array()}, /*null_count=*/0); + out_array->child_data.push_back(struct_array); + + return Status::OK(); + } +}; + +template +void AddMapCast(CastFunction* func) { + ScalarKernel kernel; + kernel.exec = CastMap::Exec; + kernel.signature = + KernelSignature::Make({InputType(MapType::type_id)}, kOutputTargetType); + kernel.null_handling = NullHandling::COMPUTED_NO_PREALLOCATE; + DCHECK_OK(func->AddKernel(MapType::type_id, std::move(kernel))); +} + } // namespace std::vector> GetNestedCasts() { @@ -253,6 +326,12 @@ std::vector> GetNestedCasts() { AddListCast(cast_large_list.get()); AddListCast(cast_large_list.get()); + auto cast_map = std::make_shared("cast_map", Type::MAP); + AddCommonCasts(Type::MAP, kOutputTargetType, cast_map.get()); + AddMapCast(cast_map.get()); + AddMapCast(cast_list.get()); + AddMapCast(cast_large_list.get()); + // FSL is a bit incomplete at the moment auto cast_fsl = std::make_shared("cast_fixed_size_list", Type::FIXED_SIZE_LIST); @@ -269,7 +348,7 @@ std::vector> GetNestedCasts() { std::make_shared("cast_dictionary", Type::DICTIONARY); AddCommonCasts(Type::DICTIONARY, kOutputTargetType, cast_dictionary.get()); - return {cast_list, cast_large_list, cast_fsl, cast_struct, cast_dictionary}; + return {cast_list, cast_large_list, cast_map, cast_fsl, cast_struct, cast_dictionary}; } } // namespace internal diff --git a/cpp/src/arrow/compute/kernels/scalar_cast_test.cc b/cpp/src/arrow/compute/kernels/scalar_cast_test.cc index 1d4780a88ebc3..e2a58a1587772 100644 --- a/cpp/src/arrow/compute/kernels/scalar_cast_test.cc +++ b/cpp/src/arrow/compute/kernels/scalar_cast_test.cc @@ -2276,6 +2276,54 @@ TEST(Cast, FSLToFSLOptionsPassThru) { CheckCast(fsl_int32, ArrayFromJSON(fixed_size_list(int16(), 1), "[[32689]]"), options); } +TEST(Cast, CastMap) { + const std::string map_json = + "[[[\"x\", 1], [\"y\", 8], [\"z\", 9]], [[\"x\", 6]], [[\"y\", 36]]]"; + const std::string map_json_nullable = + "[[[\"x\", 1], [\"y\", null], [\"z\", 9]], null, [[\"y\", 36]]]"; + + auto CheckMapCast = [map_json, + map_json_nullable](const std::shared_ptr& dst_type) { + std::shared_ptr src_type = + std::make_shared(field("x", utf8(), false), field("y", int64())); + std::shared_ptr src = ArrayFromJSON(src_type, map_json); + std::shared_ptr dst = ArrayFromJSON(dst_type, map_json); + CheckCast(src, dst); + + src = ArrayFromJSON(src_type, map_json_nullable); + dst = ArrayFromJSON(dst_type, map_json_nullable); + CheckCast(src, dst); + }; + + // Can rename fields + CheckMapCast(std::make_shared(field("a", utf8(), false), field("b", int64()))); + // Can map keys and values + CheckMapCast(map(large_utf8(), field("y", int32()))); + // Can cast a map to a to a list> + CheckMapCast(list(struct_({field("a", utf8()), field("b", int64())}))); + // Can cast a map to a large_list> + CheckMapCast(large_list(struct_({field("a", utf8()), field("b", int64())}))); + + // Can rename nested field names + std::shared_ptr src_type = map(utf8(), field("x", list(field("a", int64())))); + std::shared_ptr dst_type = map(utf8(), field("y", list(field("b", int64())))); + + std::shared_ptr src = + ArrayFromJSON(src_type, "[[[\"1\", [1,2,3]]], [[\"2\", [4,5,6]]]]"); + std::shared_ptr dst = + ArrayFromJSON(dst_type, "[[[\"1\", [1,2,3]]], [[\"2\", [4,5,6]]]]"); + + CheckCast(src, dst); + + // Cannot cast to a list> if there are not exactly 2 fields + dst_type = list( + struct_({field("key", int32()), field("value", int64()), field("extra", int64())})); + EXPECT_RAISES_WITH_MESSAGE_THAT( + TypeError, + ::testing::HasSubstr("must be cast to a list with exactly two fields"), + Cast(src, dst_type)); +} + static void CheckStructToStruct( const std::vector>& value_types) { for (const auto& src_value_type : value_types) { diff --git a/cpp/src/arrow/dataset/dataset_writer.cc b/cpp/src/arrow/dataset/dataset_writer.cc index 844f1fe2698cf..220bdad23daf6 100644 --- a/cpp/src/arrow/dataset/dataset_writer.cc +++ b/cpp/src/arrow/dataset/dataset_writer.cc @@ -131,7 +131,8 @@ class DatasetWriterFileQueue { DatasetWriterState* writer_state) : options_(options), schema_(schema), writer_state_(writer_state) {} - void Start(util::AsyncTaskScheduler* scheduler, const std::string& filename) { + void Start(std::shared_ptr scheduler, + const std::string& filename) { scheduler_ = scheduler; // Because the scheduler runs one task at a time we know the writer will // be opened before any attempt to write @@ -212,7 +213,7 @@ class DatasetWriterFileQueue { // is a 1-task FIFO we know this task will run at the very end and can // add it now. scheduler_->AddSimpleTask([this] { return DoFinish(); }); - scheduler_->End(); + scheduler_.reset(); return Status::OK(); } @@ -247,7 +248,7 @@ class DatasetWriterFileQueue { // point they are merged together and added to write_queue_ std::deque> staged_batches_; uint64_t rows_currently_staged_ = 0; - util::AsyncTaskScheduler* scheduler_ = nullptr; + std::shared_ptr scheduler_ = nullptr; }; struct WriteTask { @@ -323,17 +324,17 @@ class DatasetWriterDirectoryQueue { util::AsyncTaskScheduler::MakeThrottle(1); util::AsyncTaskScheduler::Throttle* throttle_view = throttle.get(); auto file_finish_task = [self = this, file_queue = std::move(file_queue), - throttle = std::move(throttle)]() { + throttle = std::move(throttle)](Status) { self->writer_state_->open_files_throttle.Release(1); return Status::OK(); }; - util::AsyncTaskScheduler* file_scheduler = + std::shared_ptr file_scheduler = scheduler_->MakeSubScheduler(std::move(file_finish_task), throttle_view); - if (init_task_) { - file_scheduler->AddSimpleTask(init_task_); - init_task_ = {}; + if (init_future_.is_valid()) { + file_scheduler->AddSimpleTask( + [init_future = init_future_]() { return init_future; }); } - file_queue_view->Start(file_scheduler, filename); + file_queue_view->Start(std::move(file_scheduler), filename); return file_queue_view; } @@ -343,21 +344,33 @@ class DatasetWriterDirectoryQueue { if (directory_.empty() || !write_options_.create_dir) { return; } + init_future_ = Future<>::Make(); auto create_dir_cb = [this] { return DeferNotOk(write_options_.filesystem->io_context().executor()->Submit( [this]() { return write_options_.filesystem->CreateDir(directory_); })); }; + // We need to notify waiters whether the directory succeeded or failed. + auto notify_waiters_cb = [this] { init_future_.MarkFinished(); }; + auto notify_waiters_on_err_cb = [this](const Status& err) { + init_future_.MarkFinished(); + return err; + }; + std::function()> init_task; if (write_options_.existing_data_behavior == ExistingDataBehavior::kDeleteMatchingPartitions) { - init_task_ = [this, create_dir_cb] { + init_task = [this, create_dir_cb, notify_waiters_cb, notify_waiters_on_err_cb] { return write_options_.filesystem ->DeleteDirContentsAsync(directory_, /*missing_dir_ok=*/true) - .Then(create_dir_cb); + .Then(create_dir_cb) + .Then(notify_waiters_cb, notify_waiters_on_err_cb); }; } else { - init_task_ = [create_dir_cb] { return create_dir_cb(); }; + init_task = [create_dir_cb, notify_waiters_cb, notify_waiters_on_err_cb] { + return create_dir_cb().Then(notify_waiters_cb, notify_waiters_on_err_cb); + }; } + scheduler_->AddSimpleTask(std::move(init_task)); } static Result> Make( @@ -388,7 +401,7 @@ class DatasetWriterDirectoryQueue { std::shared_ptr schema_; const FileSystemDatasetWriteOptions& write_options_; DatasetWriterState* writer_state_; - std::function()> init_task_; + Future<> init_future_; std::string current_filename_; DatasetWriterFileQueue* latest_open_file_ = nullptr; uint64_t rows_written_ = 0; diff --git a/cpp/src/arrow/dataset/dataset_writer_test.cc b/cpp/src/arrow/dataset/dataset_writer_test.cc index 6c9c292739399..e27f98dc6ef60 100644 --- a/cpp/src/arrow/dataset/dataset_writer_test.cc +++ b/cpp/src/arrow/dataset/dataset_writer_test.cc @@ -234,6 +234,18 @@ TEST_F(DatasetWriterTestFixture, BasicFileDirectoryPrefix) { AssertFilesCreated({"testdir/a/1_chunk-0.arrow"}); } +TEST_F(DatasetWriterTestFixture, DirectoryCreateFails) { + // This should fail to be created + write_options_.base_dir = "///doesnotexist"; + EXPECT_OK_AND_ASSIGN(auto dataset_writer, + DatasetWriter::Make(write_options_, scheduler_.get())); + Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), "a", "1_"); + AssertFinished(queue_fut); + ASSERT_OK(dataset_writer->Finish()); + scheduler_->End(); + ASSERT_FINISHES_AND_RAISES(Invalid, scheduler_->OnFinished()); +} + TEST_F(DatasetWriterTestFixture, MaxRowsOneWrite) { write_options_.max_rows_per_file = 10; write_options_.max_rows_per_group = 10; diff --git a/cpp/src/arrow/dataset/file_base.cc b/cpp/src/arrow/dataset/file_base.cc index a4aaaee99e914..bd19c99a52eef 100644 --- a/cpp/src/arrow/dataset/file_base.cc +++ b/cpp/src/arrow/dataset/file_base.cc @@ -484,7 +484,7 @@ class TeeNode : public compute::MapNode { util::AsyncTaskScheduler::MakeThrottle(1); util::AsyncTaskScheduler::Throttle* serial_throttle_view = serial_throttle.get(); serial_scheduler_ = plan_->async_scheduler()->MakeSubScheduler( - [owned_throttle = std::move(serial_throttle)]() { return Status::OK(); }, + [owned_throttle = std::move(serial_throttle)](Status) { return Status::OK(); }, serial_throttle_view); } @@ -519,7 +519,7 @@ class TeeNode : public compute::MapNode { MapNode::Finish(std::move(writer_finish_st)); return; } - serial_scheduler_->End(); + serial_scheduler_.reset(); MapNode::Finish(Status::OK()); } @@ -581,7 +581,7 @@ class TeeNode : public compute::MapNode { // We use a serial scheduler to submit tasks to the dataset writer. The dataset writer // only returns an unfinished future when it needs backpressure. Using a serial // scheduler here ensures we pause while we wait for backpressure to clear - util::AsyncTaskScheduler* serial_scheduler_; + std::shared_ptr serial_scheduler_; int32_t backpressure_counter_ = 0; }; diff --git a/cpp/src/arrow/dataset/scan_node.cc b/cpp/src/arrow/dataset/scan_node.cc index da397312b553e..31470bf988221 100644 --- a/cpp/src/arrow/dataset/scan_node.cc +++ b/cpp/src/arrow/dataset/scan_node.cc @@ -264,19 +264,22 @@ class ScanNode : public cp::ExecNode { ScanState* state_view = scan_state.get(); // Finish callback keeps the scan state alive until all scan tasks done struct StateHolder { - Status operator()() { return Status::OK(); } + Status operator()(Status) { return Status::OK(); } std::unique_ptr scan_state; }; - util::AsyncTaskScheduler* frag_scheduler = scan_scheduler->MakeSubScheduler( - StateHolder{std::move(scan_state)}, node->batches_throttle_.get()); + std::shared_ptr frag_scheduler = + scan_scheduler->MakeSubScheduler(StateHolder{std::move(scan_state)}, + node->batches_throttle_.get()); for (int i = 0; i < fragment_scanner->NumBatches(); i++) { node->num_batches_.fetch_add(1); frag_scheduler->AddTask(std::make_unique(node, state_view, i)); } Future<> list_and_scan_node = frag_scheduler->OnFinished(); - frag_scheduler->End(); // The "list fragments" task doesn't actually end until the fragments are // all scanned. This allows us to enforce fragment readahead. + if (--node->list_tasks_ == 0) { + node->scan_scheduler_.reset(); + } return list_and_scan_node; } @@ -313,8 +316,8 @@ class ScanNode : public cp::ExecNode { END_SPAN_ON_FUTURE_COMPLETION(span_, finished_); AsyncGenerator> frag_gen = GetFragments(options_.dataset.get(), options_.filter); - util::AsyncTaskScheduler* scan_scheduler = plan_->async_scheduler()->MakeSubScheduler( - [this]() { + scan_scheduler_ = plan_->async_scheduler()->MakeSubScheduler( + [this](Status st) { outputs_[0]->InputFinished(this, num_batches_.load()); finished_.MarkFinished(); return Status::OK(); @@ -322,12 +325,15 @@ class ScanNode : public cp::ExecNode { fragments_throttle_.get()); plan_->async_scheduler()->AddAsyncGenerator>( std::move(frag_gen), - [this, scan_scheduler](const std::shared_ptr& fragment) { - scan_scheduler->AddTask(std::make_unique(this, fragment)); + [this](const std::shared_ptr& fragment) { + list_tasks_++; + scan_scheduler_->AddTask(std::make_unique(this, fragment)); return Status::OK(); }, - [scan_scheduler]() { - scan_scheduler->End(); + [this](Status) { + if (--list_tasks_ == 0) { + scan_scheduler_.reset(); + } return Status::OK(); }); return Status::OK(); @@ -351,6 +357,11 @@ class ScanNode : public cp::ExecNode { private: ScanV2Options options_; std::atomic num_batches_{0}; + // TODO(ARROW-17509) list_tasks_, and scan_scheduler_ are just + // needed to figure out when to end scan_scheduler_. In the future, we should not need + // to call end and these variables can go away. + std::atomic list_tasks_{1}; + std::shared_ptr scan_scheduler_; std::unique_ptr fragments_throttle_; std::unique_ptr batches_throttle_; }; diff --git a/cpp/src/arrow/engine/substrait/relation_internal.cc b/cpp/src/arrow/engine/substrait/relation_internal.cc index 2afe177eb16ae..ad500aa18d851 100644 --- a/cpp/src/arrow/engine/substrait/relation_internal.cc +++ b/cpp/src/arrow/engine/substrait/relation_internal.cc @@ -96,6 +96,24 @@ Status CheckRelCommon(const RelMessage& rel) { return Status::OK(); } +Status DiscoverFilesFromDir(const std::shared_ptr& local_fs, + const std::string& dirpath, + std::vector* rel_fpaths) { + // Define a selector for a recursive descent + fs::FileSelector selector; + selector.base_dir = dirpath; + selector.recursive = true; + + ARROW_ASSIGN_OR_RAISE(auto file_infos, local_fs->GetFileInfo(selector)); + for (auto& file_info : file_infos) { + if (file_info.IsFile()) { + rel_fpaths->push_back(std::move(file_info)); + } + } + + return Status::OK(); +} + Result FromProto(const substrait::Rel& rel, const ExtensionSet& ext_set, const ConversionOptions& conversion_options) { static bool dataset_init = false; @@ -109,6 +127,7 @@ Result FromProto(const substrait::Rel& rel, const ExtensionSet& const auto& read = rel.read(); RETURN_NOT_OK(CheckRelCommon(read)); + // Get the base schema for the read relation ARROW_ASSIGN_OR_RAISE(auto base_schema, FromProto(read.base_schema(), ext_set, conversion_options)); @@ -121,8 +140,6 @@ Result FromProto(const substrait::Rel& rel, const ExtensionSet& } if (read.has_projection()) { - // NOTE: scan_options->projection is not used by the scanner and thus can't be - // used for this return Status::NotImplemented("substrait::ReadRel::projection"); } @@ -132,19 +149,23 @@ Result FromProto(const substrait::Rel& rel, const ExtensionSet& "plan contained a named table but a NamedTableProvider has not been " "configured"); } + + if (read.named_table().names().empty()) { + return Status::Invalid("names for NamedTable not provided"); + } + const NamedTableProvider& named_table_provider = conversion_options.named_table_provider; const substrait::ReadRel::NamedTable& named_table = read.named_table(); std::vector table_names(named_table.names().begin(), named_table.names().end()); - if (table_names.empty()) { - return Status::Invalid("names for NamedTable not provided"); - } ARROW_ASSIGN_OR_RAISE(compute::Declaration source_decl, named_table_provider(table_names)); + if (!source_decl.IsValid()) { return Status::Invalid("Invalid NamedTable Source"); } + return ProcessEmit(std::move(read), DeclarationInfo{std::move(source_decl), base_schema}, std::move(base_schema)); @@ -165,38 +186,7 @@ Result FromProto(const substrait::Rel& rel, const ExtensionSet& std::vector files; for (const auto& item : read.local_files().items()) { - std::string path; - if (item.path_type_case() == - substrait::ReadRel::LocalFiles::FileOrFiles::kUriPath) { - path = item.uri_path(); - } else if (item.path_type_case() == - substrait::ReadRel::LocalFiles::FileOrFiles::kUriFile) { - path = item.uri_file(); - } else if (item.path_type_case() == - substrait::ReadRel::LocalFiles::FileOrFiles::kUriFolder) { - path = item.uri_folder(); - } else { - path = item.uri_path_glob(); - } - - switch (item.file_format_case()) { - case substrait::ReadRel::LocalFiles::FileOrFiles::kParquet: - format = std::make_shared(); - break; - case substrait::ReadRel::LocalFiles::FileOrFiles::kArrow: - format = std::make_shared(); - break; - default: - return Status::NotImplemented( - "unknown substrait::ReadRel::LocalFiles::FileOrFiles::file_format"); - } - - if (!StartsWith(path, "file:///")) { - return Status::NotImplemented("substrait::ReadRel::LocalFiles item (", path, - ") with other than local filesystem " - "(file:///)"); - } - + // Validate properties of the `FileOrFiles` item if (item.partition_index() != 0) { return Status::NotImplemented( "non-default substrait::ReadRel::LocalFiles::FileOrFiles::partition_index"); @@ -212,43 +202,101 @@ Result FromProto(const substrait::Rel& rel, const ExtensionSet& "non-default substrait::ReadRel::LocalFiles::FileOrFiles::length"); } - path = path.substr(7); + // Extract and parse the read relation's source URI + ::arrow::internal::Uri item_uri; switch (item.path_type_case()) { - case substrait::ReadRel::LocalFiles::FileOrFiles::kUriPath: { - ARROW_ASSIGN_OR_RAISE(auto file, filesystem->GetFileInfo(path)); - if (file.type() == fs::FileType::File) { - files.push_back(std::move(file)); - } else if (file.type() == fs::FileType::Directory) { - fs::FileSelector selector; - selector.base_dir = path; - selector.recursive = true; - ARROW_ASSIGN_OR_RAISE(auto discovered_files, - filesystem->GetFileInfo(selector)); - std::move(files.begin(), files.end(), std::back_inserter(discovered_files)); - } + case substrait::ReadRel::LocalFiles::FileOrFiles::kUriPath: + RETURN_NOT_OK(item_uri.Parse(item.uri_path())); break; - } + + case substrait::ReadRel::LocalFiles::FileOrFiles::kUriFile: + RETURN_NOT_OK(item_uri.Parse(item.uri_file())); + break; + + case substrait::ReadRel::LocalFiles::FileOrFiles::kUriFolder: + RETURN_NOT_OK(item_uri.Parse(item.uri_folder())); + break; + + default: + RETURN_NOT_OK(item_uri.Parse(item.uri_path_glob())); + break; + } + + // Validate the URI before processing + if (!item_uri.is_file_scheme()) { + return Status::NotImplemented("substrait::ReadRel::LocalFiles item (", + item_uri.ToString(), + ") does not have file scheme (file:///)"); + } + + if (item_uri.port() != -1) { + return Status::NotImplemented("substrait::ReadRel::LocalFiles item (", + item_uri.ToString(), + ") should not have a port number in path"); + } + + if (!item_uri.query_string().empty()) { + return Status::NotImplemented("substrait::ReadRel::LocalFiles item (", + item_uri.ToString(), + ") should not have a query string in path"); + } + + switch (item.file_format_case()) { + case substrait::ReadRel::LocalFiles::FileOrFiles::kParquet: + format = std::make_shared(); + break; + case substrait::ReadRel::LocalFiles::FileOrFiles::kArrow: + format = std::make_shared(); + break; + default: + return Status::NotImplemented( + "unsupported file format ", + "(see substrait::ReadRel::LocalFiles::FileOrFiles::file_format)"); + } + + // Handle the URI as appropriate + switch (item.path_type_case()) { case substrait::ReadRel::LocalFiles::FileOrFiles::kUriFile: { - files.emplace_back(path, fs::FileType::File); + files.emplace_back(item_uri.path(), fs::FileType::File); break; } + case substrait::ReadRel::LocalFiles::FileOrFiles::kUriFolder: { - fs::FileSelector selector; - selector.base_dir = path; - selector.recursive = true; - ARROW_ASSIGN_OR_RAISE(auto discovered_files, - filesystem->GetFileInfo(selector)); - std::move(discovered_files.begin(), discovered_files.end(), - std::back_inserter(files)); + RETURN_NOT_OK(DiscoverFilesFromDir(filesystem, item_uri.path(), &files)); + break; + } + + case substrait::ReadRel::LocalFiles::FileOrFiles::kUriPath: { + ARROW_ASSIGN_OR_RAISE(auto file_info, + filesystem->GetFileInfo(item_uri.path())); + + switch (file_info.type()) { + case fs::FileType::File: { + files.push_back(std::move(file_info)); + break; + } + case fs::FileType::Directory: { + RETURN_NOT_OK(DiscoverFilesFromDir(filesystem, item_uri.path(), &files)); + break; + } + case fs::FileType::NotFound: + return Status::Invalid("Unable to find file for URI path"); + case fs::FileType::Unknown: + [[fallthrough]]; + default: + return Status::NotImplemented("URI path is of unknown file type."); + } break; } + case substrait::ReadRel::LocalFiles::FileOrFiles::kUriPathGlob: { - ARROW_ASSIGN_OR_RAISE(auto discovered_files, - fs::internal::GlobFiles(filesystem, path)); - std::move(discovered_files.begin(), discovered_files.end(), + ARROW_ASSIGN_OR_RAISE(auto globbed_files, + fs::internal::GlobFiles(filesystem, item_uri.path())); + std::move(globbed_files.begin(), globbed_files.end(), std::back_inserter(files)); break; } + default: { return Status::Invalid("Unrecognized file type in LocalFiles"); } @@ -261,7 +309,7 @@ Result FromProto(const substrait::Rel& rel, const ExtensionSet& ARROW_ASSIGN_OR_RAISE(auto ds, ds_factory->Finish(base_schema)); - DeclarationInfo scan_declaration = { + DeclarationInfo scan_declaration{ compute::Declaration{"scan", dataset::ScanNodeOptions{ds, scan_options}}, base_schema}; diff --git a/cpp/src/arrow/engine/substrait/serde_test.cc b/cpp/src/arrow/engine/substrait/serde_test.cc index 564d68f3606d5..1a13156b63412 100644 --- a/cpp/src/arrow/engine/substrait/serde_test.cc +++ b/cpp/src/arrow/engine/substrait/serde_test.cc @@ -987,9 +987,6 @@ Result GetSubstraitJSON() { } TEST(Substrait, DeserializeWithConsumerFactory) { -#ifdef _WIN32 - GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows"; -#else ASSERT_OK_AND_ASSIGN(std::string substrait_json, GetSubstraitJSON()); ASSERT_OK_AND_ASSIGN(auto buf, SerializeJsonPlan(substrait_json)); ASSERT_OK_AND_ASSIGN(auto declarations, @@ -1006,13 +1003,9 @@ TEST(Substrait, DeserializeWithConsumerFactory) { ASSERT_OK(plan->StartProducing()); ASSERT_FINISHES_OK(plan->finished()); -#endif } TEST(Substrait, DeserializeSinglePlanWithConsumerFactory) { -#ifdef _WIN32 - GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows"; -#else ASSERT_OK_AND_ASSIGN(std::string substrait_json, GetSubstraitJSON()); ASSERT_OK_AND_ASSIGN(auto buf, SerializeJsonPlan(substrait_json)); ASSERT_OK_AND_ASSIGN(std::shared_ptr plan, @@ -1026,13 +1019,9 @@ TEST(Substrait, DeserializeSinglePlanWithConsumerFactory) { ASSERT_OK(plan->StartProducing()); ASSERT_FINISHES_OK(plan->finished()); -#endif } TEST(Substrait, DeserializeWithWriteOptionsFactory) { -#ifdef _WIN32 - GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows"; -#else dataset::internal::Initialize(); fs::TimePoint mock_now = std::chrono::system_clock::now(); fs::FileInfo testdir = ::arrow::fs::Dir("testdir"); @@ -1069,7 +1058,6 @@ TEST(Substrait, DeserializeWithWriteOptionsFactory) { ASSERT_OK(plan->StartProducing()); ASSERT_FINISHES_OK(plan->finished()); -#endif } static void test_with_registries( @@ -1084,9 +1072,6 @@ static void test_with_registries( } TEST(Substrait, GetRecordBatchReader) { -#ifdef _WIN32 - GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows"; -#else ASSERT_OK_AND_ASSIGN(std::string substrait_json, GetSubstraitJSON()); test_with_registries([&substrait_json](ExtensionIdRegistry* ext_id_reg, compute::FunctionRegistry* func_registry) { @@ -1097,7 +1082,6 @@ TEST(Substrait, GetRecordBatchReader) { // in case of a test failure, re-evalaute the content in the file EXPECT_EQ(table->num_rows(), 12); }); -#endif } TEST(Substrait, InvalidPlan) { diff --git a/cpp/src/arrow/filesystem/s3_test_util.cc b/cpp/src/arrow/filesystem/s3_test_util.cc index d7e0cbc92d74f..eb29a677dae9e 100644 --- a/cpp/src/arrow/filesystem/s3_test_util.cc +++ b/cpp/src/arrow/filesystem/s3_test_util.cc @@ -111,12 +111,10 @@ Status MinioTestServer::Start() { bp::environment env = boost::this_process::environment(); env["MINIO_ACCESS_KEY"] = kMinioAccessKey; env["MINIO_SECRET_KEY"] = kMinioSecretKey; + // Disable the embedded console (one less listening address to care about) + env["MINIO_BROWSER"] = "off"; impl_->connect_string_ = GenerateConnectString(); - // Also generate a console address, as it seems that Minio sometimes - // tries to listen on a port already in use. - const auto console_address = GenerateConnectString(); - auto exe_path = bp::search_path(kMinioExecutableName); if (exe_path.empty()) { return Status::IOError("Failed to find minio executable ('", kMinioExecutableName, @@ -127,8 +125,7 @@ Status MinioTestServer::Start() { // NOTE: --quiet makes startup faster by suppressing remote version check impl_->server_process_ = std::make_shared( env, exe_path, "server", "--quiet", "--compat", "--address", - impl_->connect_string_, "--console-address", console_address, - impl_->temp_dir_->path().ToString()); + impl_->connect_string_, impl_->temp_dir_->path().ToString()); } catch (const std::exception& e) { return Status::IOError("Failed to launch Minio server: ", e.what()); } diff --git a/cpp/src/arrow/filesystem/s3fs_test.cc b/cpp/src/arrow/filesystem/s3fs_test.cc index 69ad5306a0a95..c6957fa933ec8 100644 --- a/cpp/src/arrow/filesystem/s3fs_test.cc +++ b/cpp/src/arrow/filesystem/s3fs_test.cc @@ -179,8 +179,23 @@ class S3TestMixin : public AwsTestMixin { void SetUp() override { AwsTestMixin::SetUp(); - ASSERT_OK_AND_ASSIGN(minio_, GetMinioEnv()->GetOneServer()); + // Starting the server may fail, for example if the generated port number + // was "stolen" by another process. Run a dummy S3 operation to make sure it + // is running, otherwise retry a number of times. + Status connect_status; + int retries = kNumServerRetries; + do { + InitServerAndClient(); + connect_status = OutcomeToStatus("ListBuckets", client_->ListBuckets()); + } while (!connect_status.ok() && --retries > 0); + ASSERT_OK(connect_status); + } + + void TearDown() override { AwsTestMixin::TearDown(); } + protected: + void InitServerAndClient() { + ASSERT_OK_AND_ASSIGN(minio_, GetMinioEnv()->GetOneServer()); client_config_.reset(new Aws::Client::ClientConfiguration()); client_config_->endpointOverride = ToAwsString(minio_->connect_string()); client_config_->scheme = Aws::Http::Scheme::HTTP; @@ -194,9 +209,9 @@ class S3TestMixin : public AwsTestMixin { use_virtual_addressing)); } - void TearDown() override { AwsTestMixin::TearDown(); } + // How many times to try launching a server in a row before decreeing failure + static constexpr int kNumServerRetries = 3; - protected: std::shared_ptr minio_; std::unique_ptr client_config_; Aws::Auth::AWSCredentials credentials_; diff --git a/cpp/src/arrow/util/async_generator_test.cc b/cpp/src/arrow/util/async_generator_test.cc index 4d86df34b8f92..37718f743fffc 100644 --- a/cpp/src/arrow/util/async_generator_test.cc +++ b/cpp/src/arrow/util/async_generator_test.cc @@ -705,6 +705,7 @@ TEST_P(MergedGeneratorTestFixture, MergedLimitedSubscriptions) { AssertGeneratorExhausted(merged); } +#ifndef ARROW_VALGRIND TEST_P(MergedGeneratorTestFixture, MergedStress) { constexpr int NGENERATORS = 10; constexpr int NITEMS = 10; @@ -739,6 +740,7 @@ TEST_P(MergedGeneratorTestFixture, MergedParallelStress) { ASSERT_EQ(NITEMS * NGENERATORS, items.size()); } } +#endif TEST_P(MergedGeneratorTestFixture, MergedRecursion) { // Regression test for an edge case in MergedGenerator. Ensure if diff --git a/cpp/src/arrow/util/async_util.cc b/cpp/src/arrow/util/async_util.cc index e11f299ba0789..d5b1388da8822 100644 --- a/cpp/src/arrow/util/async_util.cc +++ b/cpp/src/arrow/util/async_util.cc @@ -98,13 +98,51 @@ class FifoQueue : public AsyncTaskScheduler::Queue { std::list> tasks_; }; +class AlreadyFailedScheduler : public AsyncTaskScheduler { + public: + explicit AlreadyFailedScheduler(Status failure_reason, + FnOnce finish_callback) + : failure_reason_(std::move(failure_reason)), + finish_callback_(std::move(finish_callback)) {} + ~AlreadyFailedScheduler() override { + std::ignore = std::move(finish_callback_)(failure_reason_); + } + bool AddTask(std::unique_ptr task) override { return false; } + void End() override { + Status::UnknownError("Do not call End on a sub-scheduler.").Abort(); + } + Future<> OnFinished() const override { + Status::UnknownError( + "You should not rely on sub-scheduler's OnFinished. Use a " + "finished callback when creating the sub-scheduler instead") + .Abort(); + } + std::shared_ptr MakeSubScheduler( + FnOnce finish_callback, Throttle* throttle, + std::unique_ptr queue) override { + return AlreadyFailedScheduler::Make(failure_reason_, std::move(finish_callback)); + } + static std::unique_ptr Make( + Status failure, FnOnce finish_callback) { + DCHECK(!failure.ok()); + return std::make_unique(std::move(failure), + std::move(finish_callback)); + } + // This is deleted when ended so there is no possible way for this to return true + bool IsEnded() override { return false; } + + private: + Status failure_reason_; + FnOnce finish_callback_; +}; + class AsyncTaskSchedulerImpl : public AsyncTaskScheduler { public: using Task = AsyncTaskScheduler::Task; using Queue = AsyncTaskScheduler::Queue; AsyncTaskSchedulerImpl(AsyncTaskSchedulerImpl* parent, std::unique_ptr queue, - Throttle* throttle, FnOnce finish_callback) + Throttle* throttle, FnOnce finish_callback) : AsyncTaskScheduler(), queue_(std::move(queue)), throttle_(throttle), @@ -128,6 +166,9 @@ class AsyncTaskSchedulerImpl : public AsyncTaskScheduler { Status::UnknownError("AsyncTaskScheduler abandoned before completion"), std::move(lk)); } + if (state_ != State::kEnded) { + End(/*from_destructor=*/true); + } } finished_.Wait(); } @@ -172,16 +213,27 @@ class AsyncTaskSchedulerImpl : public AsyncTaskScheduler { return true; } - AsyncTaskScheduler* MakeSubScheduler(FnOnce finish_callback, - Throttle* throttle, - std::unique_ptr queue) override { - std::unique_ptr owned_child = - std::make_unique(this, std::move(queue), throttle, - std::move(finish_callback)); - AsyncTaskScheduler* child = owned_child.get(); + bool IsEnded() override { + std::lock_guard lk(mutex_); + return state_ == State::kEnded; + } + + std::shared_ptr MakeSubScheduler( + FnOnce finish_callback, Throttle* throttle, + std::unique_ptr queue) override { + AsyncTaskSchedulerImpl* child; std::list>::iterator child_itr; { std::lock_guard lk(mutex_); + DCHECK_NE(state_, State::kEnded) + << "Attempt to create a sub-scheduler on an ended parent."; + if (state_ != State::kRunning) { + return AlreadyFailedScheduler::Make(maybe_error_, std::move(finish_callback)); + } + std::unique_ptr owned_child = + std::make_unique(this, std::move(queue), throttle, + std::move(finish_callback)); + child = owned_child.get(); running_tasks_++; sub_schedulers_.push_back(std::move(owned_child)); child_itr = --sub_schedulers_.end(); @@ -190,22 +242,17 @@ class AsyncTaskSchedulerImpl : public AsyncTaskScheduler { struct Finalizer { void operator()(const Status& st) { std::unique_lock lk(self->mutex_); - FnOnce finish_callback; + FnOnce finish_callback = + std::move((*child_itr)->finish_callback_); + self->sub_schedulers_.erase(child_itr); + lk.unlock(); + Status finish_st = std::move(finish_callback)(st); + lk.lock(); + self->running_tasks_--; if (!st.ok()) { - self->running_tasks_--; self->AbortUnlocked(st, std::move(lk)); return; - } else { - // We only eagerly erase the sub-scheduler on a successful completion. This is - // because, if the sub-scheduler aborted, then the caller of MakeSubScheduler - // might still be planning to call End - finish_callback = std::move((*child_itr)->finish_callback_); - self->sub_schedulers_.erase(child_itr); } - lk.unlock(); - Status finish_st = std::move(finish_callback)(); - lk.lock(); - self->running_tasks_--; if (!finish_st.ok()) { self->AbortUnlocked(finish_st, std::move(lk)); return; @@ -221,14 +268,16 @@ class AsyncTaskSchedulerImpl : public AsyncTaskScheduler { }; child->OnFinished().AddCallback(Finalizer{this, child_itr}); - return child; + return CreateEndingHolder(child); } - void End() override { - std::unique_lock lk(mutex_); - if (state_ == State::kAborted) { - return; + void End() override { End(/*from_destrutor=*/false); } + + void End(bool from_destructor) { + if (!from_destructor && finish_callback_) { + Status::UnknownError("Do not call End on a sub-scheduler.").Abort(); } + std::unique_lock lk(mutex_); state_ = State::kEnded; if (running_tasks_ == 0 && (!queue_ || queue_->Empty())) { lk.unlock(); @@ -239,6 +288,16 @@ class AsyncTaskSchedulerImpl : public AsyncTaskScheduler { Future<> OnFinished() const override { return finished_; } private: + std::shared_ptr CreateEndingHolder( + AsyncTaskSchedulerImpl* target) { + struct SchedulerEnder { + void operator()(AsyncTaskSchedulerImpl* scheduler) { + scheduler->End(/*from_destructor=*/true); + } + }; + return std::shared_ptr(target, SchedulerEnder()); + } + void ContinueTasksUnlocked(std::unique_lock* lk) { while (!queue_->Empty()) { int next_cost = std::min(queue_->Peek().cost(), throttle_->Capacity()); @@ -267,8 +326,7 @@ class AsyncTaskSchedulerImpl : public AsyncTaskScheduler { } bool IsFullyFinished() { - return state_ != State::kRunning && (!queue_ || queue_->Empty()) && - running_tasks_ == 0; + return state_ == State::kEnded && (!queue_ || queue_->Empty()) && running_tasks_ == 0; } bool OnTaskFinished(const Status& st, int task_cost) { @@ -307,8 +365,8 @@ class AsyncTaskSchedulerImpl : public AsyncTaskScheduler { } Result> submit_result = (*task)(this); if (!submit_result.ok()) { - global_abort_->store(true); std::unique_lock lk(mutex_); + global_abort_->store(true); running_tasks_--; AbortUnlocked(submit_result.status(), std::move(lk)); return false; @@ -337,9 +395,11 @@ class AsyncTaskSchedulerImpl : public AsyncTaskScheduler { maybe_error_ = st; } } - if (running_tasks_ == 0) { + if (running_tasks_ == 0 && state_ == State::kEnded) { + lk.unlock(); + finished_.MarkFinished(maybe_error_); + } else { lk.unlock(); - finished_.MarkFinished(std::move(maybe_error_)); } } @@ -353,7 +413,7 @@ class AsyncTaskSchedulerImpl : public AsyncTaskScheduler { std::unique_ptr queue_; Throttle* throttle_; - FnOnce finish_callback_; + FnOnce finish_callback_; Future<> finished_ = Future<>::Make(); int running_tasks_ = 0; @@ -375,7 +435,7 @@ class AsyncTaskSchedulerImpl : public AsyncTaskScheduler { std::unique_ptr AsyncTaskScheduler::Make( Throttle* throttle, std::unique_ptr queue) { return std::make_unique(nullptr, std::move(queue), throttle, - FnOnce()); + FnOnce()); } } // namespace util diff --git a/cpp/src/arrow/util/async_util.h b/cpp/src/arrow/util/async_util.h index e636eb805dec5..e0b6cfc699127 100644 --- a/cpp/src/arrow/util/async_util.h +++ b/cpp/src/arrow/util/async_util.h @@ -69,7 +69,7 @@ namespace util { /// the final task future. /// /// It is also possible to limit the number of concurrent tasks the scheduler will -/// execute. This is done by setting a task limit. The task limit initially assumes all +/// execute. This is done by setting a throttle. The throttle initially assumes all /// tasks are equal but a custom cost can be supplied when scheduling a task (e.g. based /// on the total I/O cost of the task, or the expected RAM utilization of the task) /// @@ -88,7 +88,7 @@ class ARROW_EXPORT AsyncTaskScheduler { /// Destructor for AsyncTaskScheduler /// /// If a scheduler is not in the ended state when it is destroyed then it - /// will enter an aborted state. + /// will abort with an error and enter the ended state. /// /// The destructor will block until all submitted tasks have finished. virtual ~AsyncTaskScheduler() = default; @@ -170,14 +170,14 @@ class ARROW_EXPORT AsyncTaskScheduler { /// If the scheduler is in an aborted state this call will return false and the task /// will never be run. This is harmless and does not need to be guarded against. /// - /// If the scheduler is in an ended state then this call will cause an abort. This - /// represents a logic error in the program and should be avoidable. + /// If the scheduler is in an ended state then this call will cause an program abort. + /// This represents a logic error in the program and should be avoidable. /// /// If there are no limits on the number of concurrent tasks then the submit function /// will be run immediately. /// - /// Otherwise, if there is a limit to the number of concurrent tasks, then this task - /// will be inserted into the scheduler's queue and submitted when there is space. + /// Otherwise, if there is a throttle, and it is full, then this task will be inserted + /// into the scheduler's queue and submitted when there is space. /// /// The return value for this call can usually be ignored. There is little harm in /// attempting to add tasks to an aborted scheduler. It is only included for callers @@ -204,39 +204,41 @@ class ARROW_EXPORT AsyncTaskScheduler { template bool AddAsyncGenerator(std::function()> generator, std::function visitor, - FnOnce finish_callback) { - AsyncTaskScheduler* generator_scheduler = + FnOnce finish_callback) { + std::shared_ptr generator_scheduler = MakeSubScheduler(std::move(finish_callback)); struct State { - State(std::function()> generator, std::function visitor) - : generator(std::move(generator)), visitor(std::move(visitor)) {} + State(std::function()> generator, std::function visitor, + std::shared_ptr scheduler) + : generator(std::move(generator)), + visitor(std::move(visitor)), + scheduler(std::move(scheduler)) {} std::function()> generator; std::function visitor; + std::shared_ptr scheduler; }; - std::unique_ptr state_holder = - std::make_unique(std::move(generator), std::move(visitor)); + std::unique_ptr state_holder = std::make_unique( + std::move(generator), std::move(visitor), generator_scheduler); struct SubmitTask : public Task { explicit SubmitTask(std::unique_ptr state_holder) : state_holder(std::move(state_holder)) {} struct SubmitTaskCallback { - SubmitTaskCallback(AsyncTaskScheduler* scheduler, - std::unique_ptr state_holder) - : scheduler(scheduler), state_holder(std::move(state_holder)) {} + explicit SubmitTaskCallback(std::unique_ptr state_holder) + : state_holder(std::move(state_holder)) {} Status operator()(const T& item) { if (IsIterationEnd(item)) { - scheduler->End(); return Status::OK(); } ARROW_RETURN_NOT_OK(state_holder->visitor(item)); - scheduler->AddTask(std::make_unique(std::move(state_holder))); + state_holder->scheduler->AddTask( + std::make_unique(std::move(state_holder))); return Status::OK(); } - AsyncTaskScheduler* scheduler; std::unique_ptr state_holder; }; Result> operator()(AsyncTaskScheduler* scheduler) { Future next = state_holder->generator(); - return next.Then(SubmitTaskCallback(scheduler, std::move(state_holder))); + return next.Then(SubmitTaskCallback(std::move(state_holder))); } std::unique_ptr state_holder; }; @@ -259,7 +261,8 @@ class ARROW_EXPORT AsyncTaskScheduler { } /// Signal that tasks are done being added /// - /// If the scheduler is in an aborted state then this call will have no effect. + /// If the scheduler is in an aborted state then this call will have no effect + /// except (if there are no running tasks) potentially finishing the scheduler. /// /// Otherwise, this will transition the scheduler into the ended state. Once all /// remaining tasks have finished the OnFinished future will be marked completed. @@ -268,13 +271,6 @@ class ARROW_EXPORT AsyncTaskScheduler { /// attempt to do so will cause an abort. virtual void End() = 0; /// A future that will be finished after End is called and all tasks have completed - /// - /// This is the same future that is returned by End() but calling this method does - /// not indicate that top level tasks are done being added. End() must still be called - /// at some point or the future returned will never finish. - /// - /// This is a utility method for workflows where the finish future needs to be - /// referenced before all top level tasks have been queued. virtual Future<> OnFinished() const = 0; /// Create a sub-scheduler for tracking a subset of tasks @@ -292,15 +288,16 @@ class ARROW_EXPORT AsyncTaskScheduler { /// /// If either the parent scheduler or the sub-scheduler encounter an error /// then they will both enter an aborted state (this is a shared state). - /// Finish callbacks will not be run when the scheduler is aborted. + /// Finish callbacks will always be run and only when the sub-scheduler + /// has been ended and all ongoing tasks completed. /// /// The parent scheduler will not complete until the sub-scheduler's /// tasks (and finish callback) have all executed. /// /// A sub-scheduler can share the same throttle as its parent but it /// can also have its own unique throttle. - virtual AsyncTaskScheduler* MakeSubScheduler( - FnOnce finish_callback, Throttle* throttle = NULLPTR, + virtual std::shared_ptr MakeSubScheduler( + FnOnce finish_callback, Throttle* throttle = NULLPTR, std::unique_ptr queue = NULLPTR) = 0; /// Construct a scheduler @@ -311,6 +308,13 @@ class ARROW_EXPORT AsyncTaskScheduler { /// The default (nullptr) will use a FIFO queue if there is a throttle. static std::unique_ptr Make(Throttle* throttle = NULLPTR, std::unique_ptr queue = NULLPTR); + + /// Check to see if the scheduler is currently ended + /// + /// This method is primarily for testing purposes and won't normally need to be + /// called to use the scheduler. Note that a return value of false is not conclusive as + /// the scheduler may end immediately after the call. + virtual bool IsEnded() = 0; }; } // namespace util diff --git a/cpp/src/arrow/util/async_util_test.cc b/cpp/src/arrow/util/async_util_test.cc index ea03b5431dffd..5131a467c6f4a 100644 --- a/cpp/src/arrow/util/async_util_test.cc +++ b/cpp/src/arrow/util/async_util_test.cc @@ -82,11 +82,11 @@ TEST(AsyncTaskScheduler, Abandoned) { // submit any pending tasks. bool submitted_task_finished = false; bool pending_task_submitted = false; + AsyncTaskScheduler* weak_scheduler; std::unique_ptr throttle = AsyncTaskScheduler::MakeThrottle(1); Future<> finished_fut; Future<> first_task = Future<>::Make(); - AsyncTaskScheduler* weak_scheduler; std::thread delete_scheduler_thread; { std::unique_ptr scheduler = @@ -113,10 +113,7 @@ TEST(AsyncTaskScheduler, Abandoned) { } // Here we are waiting for the scheduler to enter aborted state. Once aborted the // scheduler will no longer accept new tasks and will return false. - BusyWait(10, [&] { - SleepABit(); - return !weak_scheduler->AddSimpleTask([] { return Future<>::MakeFinished(); }); - }); + BusyWait(10, [&] { return weak_scheduler->IsEnded(); }); // Now that the scheduler deletion is in progress we should be able to finish the // first task and be confident the second task should not be submitted. first_task.MarkFinished(); @@ -157,56 +154,138 @@ TEST(AsyncTaskScheduler, TaskFailsAfterEnd) { ASSERT_FINISHES_AND_RAISES(Invalid, scheduler->OnFinished()); } +FnOnce EmptyFinishCallback() { + return [](Status) { return Status::OK(); }; +} + +#ifndef ARROW_VALGRIND +TEST(AsyncTaskScheduler, FailingTaskStress) { + // Test many tasks failing at the same time + constexpr int kNumTasks = 256; + for (int i = 0; i < kNumTasks; i++) { + std::unique_ptr scheduler = AsyncTaskScheduler::Make(); + ASSERT_TRUE(scheduler->AddSimpleTask([] { return SleepABitAsync(); })); + ASSERT_TRUE(scheduler->AddSimpleTask( + [] { return SleepABitAsync().Then([]() { return Status::Invalid("XYZ"); }); })); + scheduler->End(); + ASSERT_FINISHES_AND_RAISES(Invalid, scheduler->OnFinished()); + } + for (int i = 0; i < kNumTasks; i++) { + std::unique_ptr scheduler = AsyncTaskScheduler::Make(); + { + std::shared_ptr sub_scheduler = + scheduler->MakeSubScheduler(EmptyFinishCallback()); + ASSERT_TRUE(sub_scheduler->AddSimpleTask([] { return SleepABitAsync(); })); + ASSERT_TRUE(sub_scheduler->AddSimpleTask( + [] { return SleepABitAsync().Then([]() { return Status::Invalid("XYZ"); }); })); + } + scheduler->End(); + ASSERT_FINISHES_AND_RAISES(Invalid, scheduler->OnFinished()); + } + // Test many schedulers failing at the same time (also a mixture of failing due + // to global abort racing with local task failure) + constexpr int kNumSchedulers = 32; + for (int i = 0; i < kNumTasks; i++) { + std::unique_ptr scheduler = AsyncTaskScheduler::Make(); + std::vector> tests; + for (int i = 0; i < kNumSchedulers; i++) { + tests.push_back(SleepABitAsync().Then([&] { + std::shared_ptr sub_scheduler = + scheduler->MakeSubScheduler(EmptyFinishCallback()); + std::ignore = sub_scheduler->AddSimpleTask([] { + return SleepABitAsync().Then([]() { return Status::Invalid("XYZ"); }); + }); + })); + } + AllComplete(std::move(tests)).Wait(); + scheduler->End(); + ASSERT_FINISHES_AND_RAISES(Invalid, scheduler->OnFinished()); + } +} +#endif + TEST(AsyncTaskScheduler, SubSchedulerFinishCallback) { bool finish_callback_ran = false; std::unique_ptr scheduler = AsyncTaskScheduler::Make(); - AsyncTaskScheduler* sub_scheduler = scheduler->MakeSubScheduler([&] { - finish_callback_ran = true; - return Status::OK(); - }); - ASSERT_FALSE(finish_callback_ran); - sub_scheduler->End(); + { + std::shared_ptr sub_scheduler = + scheduler->MakeSubScheduler([&](Status) { + finish_callback_ran = true; + return Status::OK(); + }); + ASSERT_FALSE(finish_callback_ran); + } ASSERT_TRUE(finish_callback_ran); scheduler->End(); ASSERT_FINISHES_OK(scheduler->OnFinished()); + + // Finish callback should run even if sub scheduler never starts any tasks + finish_callback_ran = false; + scheduler = AsyncTaskScheduler::Make(); + ASSERT_TRUE(scheduler->AddSimpleTask([] { return Status::Invalid("XYZ"); })); + { + std::shared_ptr sub_scheduler = + scheduler->MakeSubScheduler([&](Status) { + finish_callback_ran = true; + return Status::OK(); + }); + } + scheduler->End(); + ASSERT_FINISHES_AND_RAISES(Invalid, scheduler->OnFinished()); + ASSERT_TRUE(finish_callback_ran); + + // Finish callback should run even if scheduler aborts + finish_callback_ran = false; + scheduler = AsyncTaskScheduler::Make(); + { + std::shared_ptr sub_scheduler = + scheduler->MakeSubScheduler([&](Status) { + finish_callback_ran = true; + return Status::OK(); + }); + + ASSERT_TRUE(sub_scheduler->AddSimpleTask([] { return Status::Invalid("XYZ"); })); + } + ASSERT_TRUE(finish_callback_ran); + scheduler->End(); + ASSERT_FINISHES_AND_RAISES(Invalid, scheduler->OnFinished()); } TEST(AsyncTaskScheduler, SubSchedulerFinishAbort) { bool finish_callback_ran = false; std::unique_ptr scheduler = AsyncTaskScheduler::Make(); - AsyncTaskScheduler* sub_scheduler = scheduler->MakeSubScheduler([&] { - finish_callback_ran = true; - return Status::Invalid("XYZ"); - }); - ASSERT_FALSE(finish_callback_ran); - sub_scheduler->End(); + { + std::shared_ptr sub_scheduler = + scheduler->MakeSubScheduler([&](Status) { + finish_callback_ran = true; + return Status::Invalid("XYZ"); + }); + ASSERT_FALSE(finish_callback_ran); + } ASSERT_TRUE(finish_callback_ran); scheduler->End(); ASSERT_FINISHES_AND_RAISES(Invalid, scheduler->OnFinished()); } -FnOnce EmptyFinishCallback() { - return [] { return Status::OK(); }; -} - TEST(AsyncTaskScheduler, SubSchedulerNoticesParentAbort) { std::unique_ptr parent = AsyncTaskScheduler::Make(); std::unique_ptr child_throttle = AsyncTaskScheduler::MakeThrottle(1); - AsyncTaskScheduler* child = - parent->MakeSubScheduler(EmptyFinishCallback(), child_throttle.get()); - - Future<> task = Future<>::Make(); - bool was_submitted = false; - ASSERT_TRUE(child->AddSimpleTask([task] { return task; })); - ASSERT_TRUE(child->AddSimpleTask([&was_submitted] { - was_submitted = true; - return Future<>::MakeFinished(); - })); - ASSERT_TRUE(parent->AddSimpleTask([] { return Status::Invalid("XYZ"); })); - ASSERT_FALSE(child->AddSimpleTask([task] { return task; })); - task.MarkFinished(); - child->End(); + { + std::shared_ptr child = + parent->MakeSubScheduler(EmptyFinishCallback(), child_throttle.get()); + + Future<> task = Future<>::Make(); + bool was_submitted = false; + ASSERT_TRUE(child->AddSimpleTask([task] { return task; })); + ASSERT_TRUE(child->AddSimpleTask([&was_submitted] { + was_submitted = true; + return Future<>::MakeFinished(); + })); + ASSERT_TRUE(parent->AddSimpleTask([] { return Status::Invalid("XYZ"); })); + ASSERT_FALSE(child->AddSimpleTask([task] { return task; })); + task.MarkFinished(); + } parent->End(); ASSERT_FINISHES_AND_RAISES(Invalid, parent->OnFinished()); } @@ -215,10 +294,12 @@ TEST(AsyncTaskScheduler, SubSchedulerNoTasks) { // An unended sub-scheduler should keep the parent scheduler unfinished even if there // there are no tasks. std::unique_ptr parent = AsyncTaskScheduler::Make(); - AsyncTaskScheduler* child = parent->MakeSubScheduler(EmptyFinishCallback()); - parent->End(); - AssertNotFinished(parent->OnFinished()); - child->End(); + { + std::shared_ptr child = + parent->MakeSubScheduler(EmptyFinishCallback()); + parent->End(); + AssertNotFinished(parent->OnFinished()); + } ASSERT_FINISHES_OK(parent->OnFinished()); } @@ -389,6 +470,7 @@ TEST(AsyncTaskScheduler, PurgeUnsubmitted) { ASSERT_FALSE(was_submitted); } +#ifndef ARROW_VALGRIND TEST(AsyncTaskScheduler, FifoStress) { // Regresses an issue where adding a task, when the throttle was // just cleared, could lead to the added task being run immediately, @@ -410,6 +492,7 @@ TEST(AsyncTaskScheduler, FifoStress) { EXPECT_TRUE(middle_task_run); return Future<>::MakeFinished(); }); + task_group->End(); } } @@ -459,12 +542,11 @@ TEST(AsyncTaskScheduler, ScanningStress) { auto scan_batch = [&] { batches_scanned++; }; auto submit_scan = [&]() { return SleepABitAsync().Then(scan_batch); }; auto list_fragment = [&]() { - AsyncTaskScheduler* batch_scheduler = + std::shared_ptr batch_scheduler = listing_scheduler->MakeSubScheduler(EmptyFinishCallback(), batch_limit.get()); for (int i = 0; i < kBatchesPerFragment; i++) { ASSERT_TRUE(batch_scheduler->AddSimpleTask(submit_scan)); } - batch_scheduler->End(); if (++fragments_scanned == kNumFragments) { listing_scheduler->End(); } @@ -477,6 +559,7 @@ TEST(AsyncTaskScheduler, ScanningStress) { ASSERT_EQ(kExpectedBatchesScanned, batches_scanned.load()); } } +#endif class TaskWithPriority : public AsyncTaskScheduler::Task { public: diff --git a/cpp/src/arrow/util/reflection_internal.h b/cpp/src/arrow/util/reflection_internal.h index 2e994aa4b7005..d7de913bafd88 100644 --- a/cpp/src/arrow/util/reflection_internal.h +++ b/cpp/src/arrow/util/reflection_internal.h @@ -27,29 +27,6 @@ namespace arrow { namespace internal { -template -struct index_sequence {}; - -template -struct make_index_sequence_impl; - -template -using make_index_sequence = typename make_index_sequence_impl::type; - -template -using index_sequence_for = make_index_sequence; - -template -struct make_index_sequence_impl { - using type = index_sequence; -}; - -template -struct make_index_sequence_impl : make_index_sequence_impl {}; - -static_assert(std::is_same, make_index_sequence<0>>::value, ""); -static_assert(std::is_same, make_index_sequence<3>>::value, ""); - template struct all_same : std::true_type {}; @@ -63,13 +40,14 @@ template struct all_same : std::false_type {}; template -void ForEachTupleMemberImpl(const std::tuple& tup, Fn&& fn, index_sequence) { - (void)std::make_tuple((fn(std::get(tup), I), std::ignore)...); +void ForEachTupleMemberImpl(const std::tuple& tup, Fn&& fn, + std::index_sequence) { + (..., fn(std::get(tup), I)); } template void ForEachTupleMember(const std::tuple& tup, Fn&& fn) { - ForEachTupleMemberImpl(tup, fn, index_sequence_for()); + ForEachTupleMemberImpl(tup, fn, std::index_sequence_for()); } template diff --git a/cpp/src/arrow/util/type_traits.h b/cpp/src/arrow/util/type_traits.h index 80cc6297e39df..c1906152423c9 100644 --- a/cpp/src/arrow/util/type_traits.h +++ b/cpp/src/arrow/util/type_traits.h @@ -42,45 +42,5 @@ template struct is_null_pointer : std::is_same::type> { }; -#ifdef __GLIBCXX__ - -// A aligned_union backport, because old libstdc++ versions don't include it. - -constexpr std::size_t max_size(std::size_t a, std::size_t b) { return (a > b) ? a : b; } - -template -struct max_size_traits; - -template -struct max_size_traits { - static constexpr std::size_t max_sizeof() { - return max_size(sizeof(H), max_size_traits::max_sizeof()); - } - static constexpr std::size_t max_alignof() { - return max_size(alignof(H), max_size_traits::max_alignof()); - } -}; - -template <> -struct max_size_traits<> { - static constexpr std::size_t max_sizeof() { return 0; } - static constexpr std::size_t max_alignof() { return 0; } -}; - -template -struct aligned_union { - static constexpr std::size_t alignment_value = max_size_traits::max_alignof(); - static constexpr std::size_t size_value = - max_size(Len, max_size_traits::max_sizeof()); - using type = typename std::aligned_storage::type; -}; - -#else - -template -using aligned_union = std::aligned_union; - -#endif - } // namespace internal } // namespace arrow diff --git a/cpp/src/arrow/util/uri.cc b/cpp/src/arrow/util/uri.cc index ced1b18404c64..88e366fca9bf6 100644 --- a/cpp/src/arrow/util/uri.cc +++ b/cpp/src/arrow/util/uri.cc @@ -151,6 +151,8 @@ Uri& Uri::operator=(Uri&& u) { std::string Uri::scheme() const { return TextRangeToString(impl_->uri_.scheme); } +bool Uri::is_file_scheme() const { return impl_->is_file_uri_; } + std::string Uri::host() const { return TextRangeToString(impl_->uri_.hostText); } bool Uri::has_host() const { return IsTextRangeSet(impl_->uri_.hostText); } diff --git a/cpp/src/arrow/util/uri.h b/cpp/src/arrow/util/uri.h index 10853b4b7778e..24678b12dcf35 100644 --- a/cpp/src/arrow/util/uri.h +++ b/cpp/src/arrow/util/uri.h @@ -45,6 +45,9 @@ class ARROW_EXPORT Uri { /// explicit scheme. std::string scheme() const; + /// Convenience function that returns true if the scheme() is "file" + bool is_file_scheme() const; + /// Whether the URI has an explicit host name. This may return true if /// the URI has an empty host (e.g. "file:///tmp/foo"), while it returns /// false is the URI has not host component at all (e.g. "file:/tmp/foo"). diff --git a/cpp/src/arrow/util/uri_test.cc b/cpp/src/arrow/util/uri_test.cc index 8cf93b2331d68..03260336551f3 100644 --- a/cpp/src/arrow/util/uri_test.cc +++ b/cpp/src/arrow/util/uri_test.cc @@ -255,8 +255,9 @@ TEST(Uri, FileScheme) { // https://tools.ietf.org/html/rfc8089 Uri uri; - auto check_no_host = [&](std::string uri_string, std::string path) -> void { + auto check_file_no_host = [&](std::string uri_string, std::string path) -> void { ASSERT_OK(uri.Parse(uri_string)); + ASSERT_TRUE(uri.is_file_scheme()); ASSERT_EQ(uri.scheme(), "file"); ASSERT_EQ(uri.host(), ""); ASSERT_EQ(uri.path(), path); @@ -264,9 +265,20 @@ TEST(Uri, FileScheme) { ASSERT_EQ(uri.password(), ""); }; - auto check_with_host = [&](std::string uri_string, std::string host, - std::string path) -> void { + auto check_notfile_no_host = [&](std::string uri_string, std::string path) -> void { ASSERT_OK(uri.Parse(uri_string)); + ASSERT_FALSE(uri.is_file_scheme()); + ASSERT_NE(uri.scheme(), "file"); + ASSERT_EQ(uri.host(), ""); + ASSERT_EQ(uri.path(), path); + ASSERT_EQ(uri.username(), ""); + ASSERT_EQ(uri.password(), ""); + }; + + auto check_file_with_host = [&](std::string uri_string, std::string host, + std::string path) -> void { + ASSERT_OK(uri.Parse(uri_string)); + ASSERT_TRUE(uri.is_file_scheme()); ASSERT_EQ(uri.scheme(), "file"); ASSERT_EQ(uri.host(), host); ASSERT_EQ(uri.path(), path); @@ -280,16 +292,19 @@ TEST(Uri, FileScheme) { // Absolute paths // (no authority) - check_no_host("file:/", "/"); - check_no_host("file:/foo/bar", "/foo/bar"); + check_file_no_host("file:/", "/"); + check_file_no_host("file:/foo1/bar", "/foo1/bar"); // (empty authority) - check_no_host("file:///", "/"); - check_no_host("file:///foo/bar", "/foo/bar"); + check_file_no_host("file:///", "/"); + check_file_no_host("file:///foo2/bar", "/foo2/bar"); + // (not file scheme) + check_notfile_no_host("s3:/", "/"); + check_notfile_no_host("s3:///foo3/bar", "/foo3/bar"); // (non-empty authority) - check_with_host("file://localhost/", "localhost", "/"); - check_with_host("file://localhost/foo/bar", "localhost", "/foo/bar"); - check_with_host("file://hostname.com/", "hostname.com", "/"); - check_with_host("file://hostname.com/foo/bar", "hostname.com", "/foo/bar"); + check_file_with_host("file://localhost/", "localhost", "/"); + check_file_with_host("file://localhost/foo/bar", "localhost", "/foo/bar"); + check_file_with_host("file://hostname.com/", "hostname.com", "/"); + check_file_with_host("file://hostname.com/foo/bar", "hostname.com", "/foo/bar"); #ifdef _WIN32 // Relative paths @@ -298,14 +313,17 @@ TEST(Uri, FileScheme) { // Absolute paths // (no authority) - check_no_host("file:/C:/", "C:/"); - check_no_host("file:/C:/foo/bar", "C:/foo/bar"); + check_file_no_host("file:/C:/", "C:/"); + check_file_no_host("file:/C:/foo/bar", "C:/foo/bar"); // (empty authority) - check_no_host("file:///C:/", "C:/"); - check_no_host("file:///C:/foo/bar", "C:/foo/bar"); + check_file_no_host("file:///D:/", "D:/"); + check_file_no_host("file:///D:/foo/bar", "D:/foo/bar"); + // (not file scheme; so slash is prepended) + check_notfile_no_host("hive:///E:/", "/E:/"); + check_notfile_no_host("hive:/E:/foo/bar", "/E:/foo/bar"); // (non-empty authority) - check_with_host("file://server/share/", "server", "/share/"); - check_with_host("file://server/share/foo/bar", "server", "/share/foo/bar"); + check_file_with_host("file://server/share/", "server", "/share/"); + check_file_with_host("file://server/share/foo/bar", "server", "/share/foo/bar"); #endif } diff --git a/cpp/src/parquet/CMakeLists.txt b/cpp/src/parquet/CMakeLists.txt index f20a83e212465..dc55ab158d844 100644 --- a/cpp/src/parquet/CMakeLists.txt +++ b/cpp/src/parquet/CMakeLists.txt @@ -200,7 +200,7 @@ if(ARROW_HAVE_RUNTIME_AVX2) set_source_files_properties(level_conversion_bmi2.cc PROPERTIES SKIP_PRECOMPILE_HEADERS ON COMPILE_FLAGS - "${ARROW_AVX2_FLAG} -DARROW_HAVE_BMI2 ${CXX_FLAGS_RELEASE}" + "${ARROW_AVX2_FLAG} -DARROW_HAVE_BMI2 -mbmi2 ${CXX_FLAGS_RELEASE}" ) endif() diff --git a/cpp/src/parquet/encoding.cc b/cpp/src/parquet/encoding.cc index 8c0e9d98e1238..44f762d7113e4 100644 --- a/cpp/src/parquet/encoding.cc +++ b/cpp/src/parquet/encoding.cc @@ -1146,11 +1146,13 @@ int PlainDecoder::Decode(T* buffer, int max_values) { return max_values; } -class PlainBooleanDecoder : public DecoderImpl, virtual public TypedDecoder { +class PlainBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder { public: explicit PlainBooleanDecoder(const ColumnDescriptor* descr); void SetData(int num_values, const uint8_t* data, int len) override; + // Two flavors of bool decoding + int Decode(uint8_t* buffer, int max_values) override; int Decode(bool* buffer, int max_values) override; int DecodeArrow(int num_values, int null_count, const uint8_t* valid_bits, int64_t valid_bits_offset, @@ -1201,6 +1203,24 @@ inline int PlainBooleanDecoder::DecodeArrow( ParquetException::NYI("dictionaries of BooleanType"); } +int PlainBooleanDecoder::Decode(uint8_t* buffer, int max_values) { + max_values = std::min(max_values, num_values_); + bool val; + ::arrow::internal::BitmapWriter bit_writer(buffer, 0, max_values); + for (int i = 0; i < max_values; ++i) { + if (!bit_reader_->GetValue(1, &val)) { + ParquetException::EofException(); + } + if (val) { + bit_writer.Set(); + } + bit_writer.Next(); + } + bit_writer.Finish(); + num_values_ -= max_values; + return max_values; +} + int PlainBooleanDecoder::Decode(bool* buffer, int max_values) { max_values = std::min(max_values, num_values_); if (bit_reader_->GetBatch(1, buffer, max_values) != max_values) { @@ -2336,7 +2356,7 @@ class DeltaLengthByteArrayDecoder : public DecoderImpl, // ---------------------------------------------------------------------- // RLE_BOOLEAN_DECODER -class RleBooleanDecoder : public DecoderImpl, virtual public TypedDecoder { +class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder { public: explicit RleBooleanDecoder(const ColumnDescriptor* descr) : DecoderImpl(descr, Encoding::RLE) {} @@ -2372,6 +2392,10 @@ class RleBooleanDecoder : public DecoderImpl, virtual public TypedDecoder::Accumulator* out) override { diff --git a/cpp/src/parquet/encoding.h b/cpp/src/parquet/encoding.h index c32da99793de5..374a02cf491f9 100644 --- a/cpp/src/parquet/encoding.h +++ b/cpp/src/parquet/encoding.h @@ -65,7 +65,7 @@ using FLBAEncoder = TypedEncoder; template class TypedDecoder; -using BooleanDecoder = TypedDecoder; +class BooleanDecoder; using Int32Decoder = TypedDecoder; using Int64Decoder = TypedDecoder; using Int96Decoder = TypedDecoder; @@ -394,6 +394,20 @@ class DictDecoder : virtual public TypedDecoder { // ---------------------------------------------------------------------- // TypedEncoder specializations, traits, and factory functions +class BooleanDecoder : virtual public TypedDecoder { + public: + using TypedDecoder::Decode; + + /// \brief Decode and bit-pack values into a buffer + /// + /// \param[in] buffer destination for decoded values + /// This buffer will contain bit-packed values. + /// \param[in] max_values max values to decode. + /// \return The number of values decoded. Should be identical to max_values except + /// at the end of the current data page. + virtual int Decode(uint8_t* buffer, int max_values) = 0; +}; + class FLBADecoder : virtual public TypedDecoder { public: using TypedDecoder::DecodeSpaced; diff --git a/cpp/src/parquet/encoding_test.cc b/cpp/src/parquet/encoding_test.cc index 3d18cd91f033c..7d42e3e8ce315 100644 --- a/cpp/src/parquet/encoding_test.cc +++ b/cpp/src/parquet/encoding_test.cc @@ -52,7 +52,7 @@ namespace parquet { namespace test { -TEST(VectorBooleanTest, TestEncodeDecode) { +TEST(VectorBooleanTest, TestEncodeBoolDecode) { // PARQUET-454 const int nvalues = 10000; bool decode_buffer[nvalues] = {false}; @@ -82,6 +82,38 @@ TEST(VectorBooleanTest, TestEncodeDecode) { } } +TEST(VectorBooleanTest, TestEncodeIntDecode) { + // PARQUET-454 + int nvalues = 10000; + + int nbytes = static_cast(bit_util::BytesForBits(nvalues)); + + std::vector draws; + ::arrow::random_is_valid(nvalues, 0.5 /* null prob */, &draws, 0 /* seed */); + + std::unique_ptr encoder = + MakeTypedEncoder(Encoding::PLAIN); + encoder->Put(draws, nvalues); + + std::unique_ptr decoder = + MakeTypedDecoder(Encoding::PLAIN); + + std::shared_ptr encode_buffer = encoder->FlushValues(); + ASSERT_EQ(nbytes, encode_buffer->size()); + + std::vector decode_buffer(nbytes); + const uint8_t* decode_data = &decode_buffer[0]; + + decoder->SetData(nvalues, encode_buffer->data(), + static_cast(encode_buffer->size())); + int values_decoded = decoder->Decode(&decode_buffer[0], nvalues); + ASSERT_EQ(nvalues, values_decoded); + + for (int i = 0; i < nvalues; ++i) { + ASSERT_EQ(draws[i], ::arrow::bit_util::GetBit(decode_data, i)) << i; + } +} + // ---------------------------------------------------------------------- // test data generation diff --git a/cpp/thirdparty/versions.txt b/cpp/thirdparty/versions.txt index 5c04756d46c91..ba560e8f41e81 100644 --- a/cpp/thirdparty/versions.txt +++ b/cpp/thirdparty/versions.txt @@ -68,8 +68,8 @@ ARROW_OPENTELEMETRY_BUILD_VERSION=v1.4.1 ARROW_OPENTELEMETRY_BUILD_SHA256_CHECKSUM=301b1ab74a664723560f46c29f228360aff1e2d63e930b963755ea077ae67524 ARROW_OPENTELEMETRY_PROTO_BUILD_VERSION=v0.17.0 ARROW_OPENTELEMETRY_PROTO_BUILD_SHA256_CHECKSUM=f269fbcb30e17b03caa1decd231ce826e59d7651c0f71c3b28eb5140b4bb5412 -ARROW_ORC_BUILD_VERSION=1.7.6 -ARROW_ORC_BUILD_SHA256_CHECKSUM=a75e0cccaaf5e03f1699bb804f640246bec4d134cb57957125707b0b2822160d +ARROW_ORC_BUILD_VERSION=1.8.0 +ARROW_ORC_BUILD_SHA256_CHECKSUM=859d78bfded98405c32ccb2847b565a57bcc53f473a74087c1f750aeb5932e62 ARROW_PROTOBUF_BUILD_VERSION=v21.3 ARROW_PROTOBUF_BUILD_SHA256_CHECKSUM=2f723218f6cb709ae4cdc4fb5ed56a5951fc5d466f0128ce4c946b8c78c8c49f # Because of https://github.com/Tencent/rapidjson/pull/1323, we require diff --git a/dev/release/post-11-bump-versions-test.rb b/dev/release/post-11-bump-versions-test.rb index 7f7def52800f4..805abb4a1c4ca 100644 --- a/dev/release/post-11-bump-versions-test.rb +++ b/dev/release/post-11-bump-versions-test.rb @@ -208,11 +208,9 @@ def test_version_post_tag hunks: [ [ "-module github.com/apache/arrow/go/v#{@snapshot_major_version}/arrow/compute", - "+module github.com/apache/arrow/go/v#{@next_major_version}/arrow/compute", - "-replace github.com/apache/arrow/go/v#{@snapshot_major_version} => ../../", - "+replace github.com/apache/arrow/go/v#{@next_major_version} => ../../", - "-\tgithub.com/apache/arrow/go/v#{@snapshot_major_version} v#{@snapshot_major_version}.0.0-00010101000000-000000000000", - "+\tgithub.com/apache/arrow/go/v#{@next_major_version} v#{@next_major_version}.0.0-00010101000000-000000000000", + "+module github.com/apache/arrow/go/v#{@next_major_version}/arrow/compute", + "-\tgithub.com/apache/arrow/go/v#{@snapshot_major_version} v#{@release_version}", + "+\tgithub.com/apache/arrow/go/v#{@next_major_version} v#{@next_version}", ], ]} next diff --git a/dev/release/utils-prepare.sh b/dev/release/utils-prepare.sh index f6ee589f15788..997a5caa2bfe2 100644 --- a/dev/release/utils-prepare.sh +++ b/dev/release/utils-prepare.sh @@ -172,7 +172,7 @@ update_versions() { # for now we have to overcome the slight conflict between the existing # "compute" package and the new go.mod file. sed -i.bak -E -e \ - "s|v[0-9]+\\.0\\.0-00010101000000-000000000000|v${major_version}.0.0-00010101000000-000000000000|" \ + "s|arrow/go/v${major_version} v[0-9]+\\.[0-9]+\\.[0-9]+|arrow/go/v${major_version} v${version%%-*}|" \ arrow/compute/go.mod find . -name "*.bak" -exec rm {} \; diff --git a/dev/release/verify-release-candidate.sh b/dev/release/verify-release-candidate.sh index 2184d408d7eec..a9681af5287d5 100755 --- a/dev/release/verify-release-candidate.sh +++ b/dev/release/verify-release-candidate.sh @@ -836,7 +836,7 @@ test_js() { show_header "Build and test JavaScript libraries" maybe_setup_nodejs || exit 1 - maybe_setup_conda nodejs=17 || exit 1 + maybe_setup_conda nodejs=16 || exit 1 if ! command -v yarn &> /dev/null; then npm install -g yarn diff --git a/dev/tasks/conda-recipes/arrow-cpp/meta.yaml b/dev/tasks/conda-recipes/arrow-cpp/meta.yaml index 87eb034990660..e9fc89380b62d 100644 --- a/dev/tasks/conda-recipes/arrow-cpp/meta.yaml +++ b/dev/tasks/conda-recipes/arrow-cpp/meta.yaml @@ -81,8 +81,8 @@ outputs: - google-cloud-cpp # [linux] - grpc-cpp - libprotobuf - - clangdev 10 # [not (osx and arm64)] - - llvmdev 10 # [not (osx and arm64)] + - clangdev >=11 # [not (osx and arm64)] + - llvmdev >=11 # [not (osx and arm64)] - libutf8proc - lz4-c - numpy @@ -184,9 +184,9 @@ outputs: - {{ compiler("cuda") }} # [cuda_compiler_version != "None"] host: - {{ pin_subpackage('arrow-cpp', exact=True) }} - - clangdev 10 # [not (osx and arm64)] + - clangdev >=11 # [not (osx and arm64)] - cython - - llvmdev 10 # [not (osx and arm64)] + - llvmdev >=11 # [not (osx and arm64)] - numpy - openssl - python @@ -262,9 +262,9 @@ outputs: host: - {{ pin_subpackage('arrow-cpp', exact=True) }} - {{ pin_subpackage('pyarrow', exact=True) }} - - clangdev 10 # [not (osx and arm64)] + - clangdev >=11 # [not (osx and arm64)] - cython - - llvmdev 10 # [not (osx and arm64)] + - llvmdev >=11 # [not (osx and arm64)] - numpy - python - openssl diff --git a/docs/source/cpp/compute.rst b/docs/source/cpp/compute.rst index 2e3c6ab9de8c4..c8e896bd35bca 100644 --- a/docs/source/cpp/compute.rst +++ b/docs/source/cpp/compute.rst @@ -1373,22 +1373,36 @@ null input value is converted into a null output value. +-----------------------------+------------------------------------+---------+ | Extension | Extension storage type | | +-----------------------------+------------------------------------+---------+ -| List-like | List-like | \(2) | +| Struct | Struct | \(2) | ++-----------------------------+------------------------------------+---------+ +| List-like | List-like | \(3) | ++-----------------------------+------------------------------------+---------+ +| Map | Map or List of two-field struct | \(4) | +-----------------------------+------------------------------------+---------+ | Null | Any | | +-----------------------------+------------------------------------+---------+ -| Any | Extension | \(3) | +| Any | Extension | \(5) | +-----------------------------+------------------------------------+---------+ * \(1) The dictionary indices are unchanged, the dictionary values are cast from the input value type to the output value type (if a conversion is available). -* \(2) The list offsets are unchanged, the list values are cast from the +* \(2) The field names of the output type must be the same or a subset of the + field names of the input type; they also must have the same order. Casting to + a subset of field names "selects" those fields such that each output field + matches the data of the input field with the same name. + +* \(3) The list offsets are unchanged, the list values are cast from the input value type to the output value type (if a conversion is available). -* \(3) Any input type that can be cast to the resulting extension's storage type. +* \(4) Offsets are unchanged, the keys and values are cast from respective input + to output types (if a conversion is available). If output type is a list of + struct, the key field is output as the first field and the value field the + second field, regardless of field names chosen. + +* \(5) Any input type that can be cast to the resulting extension's storage type. This excludes extension types, unless being cast to the same extension type. Temporal component extraction diff --git a/docs/source/developers/python.rst b/docs/source/developers/python.rst index 188486cae3766..fc48b2d65ece9 100644 --- a/docs/source/developers/python.rst +++ b/docs/source/developers/python.rst @@ -109,7 +109,8 @@ The test groups currently include: * ``large_memory``: Test requiring a large amount of system RAM * ``orc``: Apache ORC tests * ``parquet``: Apache Parquet tests -* ``plasma``: Plasma Object Store tests +* ``plasma``: Plasma Object Store tests (deprecated since Arrow 10.0.0, + will be removed in 12.0.0 or so) * ``s3``: Tests for Amazon S3 * ``tensorflow``: Tests that involve TensorFlow @@ -330,7 +331,8 @@ adding flags with ``ON``: * ``ARROW_ORC``: Support for Apache ORC file format * ``ARROW_PARQUET``: Support for Apache Parquet file format * ``PARQUET_REQUIRE_ENCRYPTION``: Support for Parquet Modular Encryption -* ``ARROW_PLASMA``: Shared memory object store +* ``ARROW_PLASMA``: Shared memory object store (deprecated since Arrow 10.0.0, + will be removed in 12.0.0 or so) Anything set to ``ON`` above can also be turned off. Note that some compression libraries are recommended for full Parquet support. diff --git a/docs/source/format/ADBC.rst b/docs/source/format/ADBC.rst new file mode 100644 index 0000000000000..b71c8fe19fbfd --- /dev/null +++ b/docs/source/format/ADBC.rst @@ -0,0 +1,299 @@ +.. 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. + +================================= +ADBC: Arrow Database Connectivity +================================= + +Rationale +========= + +The Arrow ecosystem lacks standard database interfaces built around +Arrow data, especially for efficiently fetching large datasets +(i.e. with minimal or no serialization and copying). Without a common +API, the end result is a mix of custom protocols (e.g. BigQuery, +Snowflake) and adapters (e.g. Turbodbc_) scattered across languages. +Consumers must laboriously wrap individual systems (as `DBI is +contemplating`_ and `Trino does with connectors`_). + +ADBC aims to provide a minimal database client API standard, based on +Arrow, for C, Go, and Java (with bindings for other languages). +Applications code to this API standard (in much the same way as they +would with JDBC or ODBC), but fetch result sets in Arrow format +(e.g. via the :doc:`C Data Interface <./CDataInterface>`). They then +link to an implementation of the standard: either directly to a +vendor-supplied driver for a particular database, or to a driver +manager that abstracts across multiple drivers. Drivers implement the +standard using a database-specific API, such as Flight SQL. + +Goals +----- + +- Provide a cross-language, Arrow-based API to standardize how clients + submit queries to and fetch Arrow data from databases. +- Support both SQL dialects and the emergent `Substrait`_ standard. +- Support explicitly partitioned/distributed result sets to work + better with contemporary distributed systems. +- Allow for a variety of implementations to maximize reach. + +Non-goals +--------- + +- Replacing JDBC/ODBC in all use cases, particularly `OLTP`_ use + cases. +- Requiring or enshrining a particular database protocol for the Arrow + ecosystem. + +Example use cases +----------------- + +A C or C++ application wishes to retrieve bulk data from a Postgres +database for further analysis. The application is compiled against +the ADBC header, and executes queries via the ADBC APIs. The +application is linked against the ADBC libpq driver. At runtime, the +driver submits queries to the database via the Postgres client +libraries, and retrieves row-oriented results, which it then converts +to Arrow format before returning them to the application. + +If the application wishes to retrieve data from a database supporting +Flight SQL instead, it would link against the ADBC Flight SQL driver. +At runtime, the driver would submit queries via Flight SQL and get +back Arrow data, which is then passed unchanged and uncopied to the +application. (The application may have to edit the SQL queries, as +ADBC does not translate between SQL dialects.) + +If the application wishes to work with multiple databases, it would +link against the ADBC driver manager, and specify the desired driver +at runtime. The driver manager would pass on API calls to the correct +driver, which handles the request. + +ADBC API Standard 1.0.0 +======================= + +ADBC is a language-specific set of interface definitions that can be +implemented directly by a vendor-specific "driver" or a vendor-neutral +"driver manager". + +Version 1.0.0 of the standard corresponds to tag adbc-1.0.0 of the +repository ``apache/arrow-adbc``, which is commit +f044edf5256abfb4c091b0ad2acc73afea2c93c0_. Note that is is separate +from releases of the actual implementations. + +See the language-specific pages for details: + +.. toctree:: + :maxdepth: 1 + + ADBC/C + ADBC/Go + ADBC/Java + +Updating this specification +=========================== + +ADBC is versioned separately from the core Arrow project. The API +standard and components (driver manager, drivers) are also versioned +separately, but both follow semantic versioning. + +For example: components may make backwards-compatible releases as +1.0.0, 1.0.1, 1.1.0, 1.2.0, etc. They may release +backwards-incompatible versions such as 2.0.0, but which still +implement the API standard version 1.0.0. + +Similarly, this documentation describes the ADBC API standard version +1.0.0. If/when an ABI-compatible revision is made +(e.g. new standard options are defined), the next version would be +1.1.0. If incompatible changes are made (e.g. new API functions), the +next version would be 2.0.0. + +Related work +============ + +In the initial proposal, a survey of existing solutions and systems +was included, which is reproduced below for context, though note the +descriptions are only kept up-to-date on a best-effort basis. + +Comparison with Arrow Flight SQL +-------------------------------- + +Flight SQL is a **client-server protocol** oriented at database +developers. By implementing Flight SQL, a database can support +clients that use ADBC, JDBC, and ODBC. + +ADBC is an **API specification** oriented at database clients. By +coding to ADBC, an application can get Arrow data from a variety of +databases that use different client technologies underneath. + +Hence, the two projects complement each other. While Flight SQL +provides a client that can be used directly, we expect applications +would prefer to use ADBC instead of tying themselves to a particular +database. + +Comparison with JDBC/ODBC +------------------------- + +JDBC is a row-based API, so bridging JDBC to Arrow is hard to do +efficiently. + +ODBC provides support for bulk data with `block cursors`_, and +Turbodbc_ demonstrates that a performant Arrow-based API can be built +on top. However, it is still an awkward fit for Arrow: + +- Nulls (‘indicator’ values) are `represented as integers`_, requiring + conversion. +- `Result buffers are caller-allocated`_. This can force unnecessarily + copying data. ADBC uses the C Data Interface instead, eliminating + copies when possible (e.g. if the driver uses Flight SQL). +- Some data types are represented differently, and require + conversion. `SQL_C_BINARY`_ can sidestep this for drivers and + applications that cooperate, but then applications would have to + treat Arrow-based and non-Arrow-based data sources differently. + + - `Strings must be null-terminated`_, which would require a copy + into an Arrow array, or require that the application handle null + terminated strings in an array. + - It is implementation-defined whether strings may have embedded + nulls, but Arrow specifies UTF-8 strings for which 0x00 is a valid + byte. + - Because buffers are caller-allocated, the driver and application + must cooperate to handle large strings; `the driver must truncate + the value`_, and the application can try to fetch the value again. + - ODBC uses length buffers rather than offsets, requiring another + conversion to/from Arrow string arrays. + - `Time intervals use different representations`_. + +Hence, we think just extending ODBC is insufficient to meet the goals +of ADBC. ODBC will always be valuable for wider database support, and +providing an Arrow-based API on top of ODBC is useful. ADBC would +allow implementing/optimizing this conversion in a common library, +provide a simpler interface for consumers, and would provide an API +that Arrow-native or otherwise columnar systems can implement to +bypass this wrapper. + +.. figure:: ./ADBCQuadrants.svg + + ADBC, JDBC, and ODBC are database-agnostic. They define the + API that the application uses, but not how that API is implemented, + instead deferring to drivers to fulfill requests using the protocol + of their choice. JDBC and (generally) ODBC offer results in a + row-oriented format, while ADBC offers columnar Arrow data. + + Protocols/libraries like libpq (Postgres) and TDS (SQL Server) are + database-specific and row-oriented. Multiple databases may + implement the same protocol to try to reuse each other's work, + e.g. several databases implement the Postgres wire protocol to + benefit from its driver implementations. But the protocol itself + was not designed with multiple databases in mind, nor are they + generally meant to be used directly by applications. + + Some database-specific protocols are Arrow-native, like those of + BigQuery and ClickHouse. Flight SQL additionally is meant to be + database-agnostic, but it defines both the client-facing API and + the underlying protocol, so it's hard for applications to use it as + the API for databases that don't already implement Flight SQL. + +Existing database client APIs +----------------------------- + +:doc:`Arrow Flight SQL <./FlightSql>` + A standard building on top of Arrow Flight, defining how to use + Flight to talk to databases, retrieve metadata, execute queries, and + so on. Provides a single client in C++ and Java language that talks + to any database servers implementing the protocol. Models its API + surface (though not API design) after JDBC and ODBC. + +`DBI for R `_ + An R package/ecosystem of packages for database access. Provides a + single interface with "backends" for specific databases. While + row-oriented, `integration with Arrow is under consideration`_, + including a sketch of effectively the same idea as ADBC. + +`JDBC `_ + A Java library for database access, providing row-based + APIs. Provides a single interface with drivers for specific + databases. + +`ODBC `_ + A language-agnostic standard from the ISO/IEC for database access, + associated with Microsoft. Feature-wise, it is similar to JDBC (and + indeed JDBC can wrap ODBC drivers), but it offers columnar data + support through fetching buffers of column values. (See above for + caveats.) Provides a single C interface with drivers for specific + databases. + +`PEP 249 `_ (DBAPI 2.0) + A Python standard for database access providing row-based APIs. Not + a singular package, but rather a set of interfaces that packages + implement. + +Existing libraries +------------------ + +These are libraries which either 1) implement columnar data access for +a particular system; or 2) could be used to implement such access. + +:doc:`Arrow Flight <./Flight>` + An RPC framework optimized for transferring Arrow record batches, + with application-specific extension points but without any higher + level semantics. + +:doc:`Arrow JDBC <../java/jdbc>` + A Java submodule, part of Arrow/Java, that uses the JDBC API to + produce Arrow data. Internally, it can read data only row-at-a-time. + +`arrow-odbc `_ + A Rust community project that uses the ODBC API to produce Arrow + data, using ODBC’s buffer-based API to perform bulk copies. (See + also: Turbodbc.) + +`Arrowdantic `_ + Python bindings for an implementation of ODBC<>Arrow in Rust. + +`pgeon `_ + A client that manually parses the Postgres wire format and produces + Arrow data, bypassing JDBC/ODBC. While it attempts to optimize this + case, the Postgres wire protocol is still row-oriented. + +`Turbodbc `_ + A set of Python ODBC bindings, implementing PEP 249, that also + provides APIs to fetch data as Arrow batches, optimizing the + conversion internally. + +Papers +------ + +Raasveldt, Mark, and Hannes Mühleisen. `“Don't Hold My Data Hostage - +A Case for Client Protocol Redesign”`_. In *Proceedings of the VLDB +Endowment*, 1022–1033, 2017. + +.. External link definitions follow + +.. _f044edf5256abfb4c091b0ad2acc73afea2c93c0: https://github.com/apache/arrow-adbc/commit/f044edf5256abfb4c091b0ad2acc73afea2c93c0 +.. _arrow-adbc: https://github.com/apache/arrow-adbc +.. _block cursors: https://docs.microsoft.com/en-us/sql/odbc/reference/develop-app/block-cursors?view=sql-server-ver15 +.. _DBI is contemplating: https://r-dbi.github.io/dbi3/articles/dbi3.html +.. _“Don't Hold My Data Hostage - A Case for Client Protocol Redesign”: https://ir.cwi.nl/pub/26415 +.. _integration with Arrow is under consideration: https://r-dbi.github.io/dbi3/articles/dbi3.html#using-arrowparquet-as-an-exchange-format +.. _OLTP: https://en.wikipedia.org/wiki/Online_transaction_processing +.. _represented as integers: https://docs.microsoft.com/en-us/sql/odbc/reference/develop-app/using-length-and-indicator-values?view=sql-server-ver15 +.. _Result buffers are caller-allocated: https://docs.microsoft.com/en-us/sql/odbc/reference/develop-app/allocating-and-freeing-buffers?view=sql-server-ver15 +.. _SQL_C_BINARY: https://docs.microsoft.com/en-us/sql/odbc/reference/appendixes/transferring-data-in-its-binary-form?view=sql-server-ver15 +.. _Strings must be null-terminated: https://docs.microsoft.com/en-us/sql/odbc/reference/develop-app/character-data-and-c-strings?view=sql-server-ver15 +.. _Substrait: https://substrait.io +.. _the driver must truncate the value: https://docs.microsoft.com/en-us/sql/odbc/reference/develop-app/data-length-buffer-length-and-truncation?view=sql-server-ver15 +.. _Time intervals use different representations: https://docs.microsoft.com/en-us/sql/odbc/reference/appendixes/c-interval-structure?view=sql-server-ver15 +.. _Trino does with connectors: https://trino.io/docs/current/connector.html diff --git a/docs/source/format/ADBC/C.rst b/docs/source/format/ADBC/C.rst new file mode 100644 index 0000000000000..ee0490df36865 --- /dev/null +++ b/docs/source/format/ADBC/C.rst @@ -0,0 +1,33 @@ +.. 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. + +======================== +ADBC C API Specification +======================== + +In C, ADBC consists of a self-contained header. The header is +reproduced in full here, and is intended to be self-documenting. + +From apache/arrow-adbc commit f044edf5256abfb4c091b0ad2acc73afea2c93c0_: + +.. literalinclude:: ../../../../format/adbc.h + :language: c + :linenos: + :lineno-match: + :lines: 166-1123 + +.. _f044edf5256abfb4c091b0ad2acc73afea2c93c0: https://github.com/apache/arrow-adbc/commit/f044edf5256abfb4c091b0ad2acc73afea2c93c0 diff --git a/docs/source/format/ADBC/Go.rst b/docs/source/format/ADBC/Go.rst new file mode 100644 index 0000000000000..b94c291c62564 --- /dev/null +++ b/docs/source/format/ADBC/Go.rst @@ -0,0 +1,31 @@ +.. 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. + +========================= +ADBC Go API Specification +========================= + +In Go, ADBC consists of a set of interface definitions in the package +``github.com/apache/arrow-adbc/go/adbc``. + +Broadly, the interfaces are organized similarly to the C API +specification, and bindings to the C API can be created easily. + +See apache/arrow-adbc commit f044edf5256abfb4c091b0ad2acc73afea2c93c0_ +for the definitions. + +.. _f044edf5256abfb4c091b0ad2acc73afea2c93c0: https://github.com/apache/arrow-adbc/commit/f044edf5256abfb4c091b0ad2acc73afea2c93c0 diff --git a/docs/source/format/ADBC/Java.rst b/docs/source/format/ADBC/Java.rst new file mode 100644 index 0000000000000..a799fe07451ce --- /dev/null +++ b/docs/source/format/ADBC/Java.rst @@ -0,0 +1,33 @@ +.. 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. + +=========================== +ADBC Java API Specification +=========================== + +In Java, ADBC consists of a set of interface definitions in the +package ``org.apache.arrow.adbc:adbc-core``. + +Broadly, the interfaces are organized similarly to the C API +specification, but with conveniences for Java (actual enum +definitions, constants for common Arrow schemas, etc.) and makes use +of the Arrow Java libraries directly instead of the C Data Interface. + +See apache/arrow-adbc commit f044edf5256abfb4c091b0ad2acc73afea2c93c0_ +for the definitions. + +.. _f044edf5256abfb4c091b0ad2acc73afea2c93c0: https://github.com/apache/arrow-adbc/commit/f044edf5256abfb4c091b0ad2acc73afea2c93c0 diff --git a/docs/source/format/ADBCQuadrants.svg b/docs/source/format/ADBCQuadrants.svg new file mode 100644 index 0000000000000..6d79cf79afe0d --- /dev/null +++ b/docs/source/format/ADBCQuadrants.svg @@ -0,0 +1,64 @@ + + + + + + + + + + + + + + + + Database-specific + Database-agnostic + + + + + Arrow-native + Row-oriented + + + + ADBC + + JDBC + ODBC + + Flight SQL + BigQuery wire protocol + + libpq/Postgreswire protocol + TDS/SQL Serverwire protocol + diff --git a/docs/source/index.rst b/docs/source/index.rst index 60879993e45f5..4be72554cc828 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -80,6 +80,7 @@ target environment.** format/Integration format/CDataInterface format/CStreamInterface + format/ADBC format/Other format/Glossary diff --git a/docs/source/python/api/plasma.rst b/docs/source/python/api/plasma.rst index 8df9e4e21ac8b..0ef21116cf4f4 100644 --- a/docs/source/python/api/plasma.rst +++ b/docs/source/python/api/plasma.rst @@ -22,6 +22,10 @@ Plasma In-Memory Object Store ============================= +.. warning:: + + Plasma is deprecated since Arrow 10.0.0. It will be removed in 12.0.0 or so. + Classes ------- diff --git a/docs/source/python/getting_involved.rst b/docs/source/python/getting_involved.rst index 87fa0aca37fb2..1f05aa9088fa8 100644 --- a/docs/source/python/getting_involved.rst +++ b/docs/source/python/getting_involved.rst @@ -47,7 +47,7 @@ less complex to use. So while in some cases it might be easy to map what's in C++ to what's in Python, in many cases the C++ classes and methods are used as foundations to build easier to use entities. -.. image:: py_arch_overview.svg +.. image:: /python/images/py_arch_overview.svg :alt: Four layers of PyArrow architecture: .py, .pyx, .pxd and low level C++ code. * The ``*.py`` files in the pyarrow package are usually where the entities @@ -69,7 +69,8 @@ used as foundations to build easier to use entities. for usage in Cython. Here the C++ classes and methods are declared as they are so that in the other ``.pyx`` files they can be used to implement Python classes, functions and helpers. -* PyArrow is also based on dedicated pieces of code that are written into Arrow itself. - Those live into ``cpp/src/arrow/python`` directory and provide the low level +* Apart from Arrow C++ library, which dependence is mentioned in the previous line, + PyArrow is also based on PyArrow C++, dedicated pieces of code that live in + ``python/pyarrow/src/arrow/python`` directory and provide the low level code for capabilities like converting to and from numpy or pandas and the classes that allow to use Python objects and callbacks in C++. \ No newline at end of file diff --git a/docs/source/python/images/py_arch_overview.svg b/docs/source/python/images/py_arch_overview.svg new file mode 100644 index 0000000000000..8f0dd8838ca24 --- /dev/null +++ b/docs/source/python/images/py_arch_overview.svg @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/docs/source/python/integration/extending.rst b/docs/source/python/integration/extending.rst index 8931960f3a9eb..b380fea7e902c 100644 --- a/docs/source/python/integration/extending.rst +++ b/docs/source/python/integration/extending.rst @@ -31,7 +31,7 @@ C++ API .. default-domain:: cpp -The Arrow C++ header files are bundled with a pyarrow installation. +The Arrow C++ and PyArrow C++ header files are bundled with a pyarrow installation. To get the absolute path to this directory (like ``numpy.get_include()``), use: .. code-block:: python @@ -50,12 +50,19 @@ This will not include other parts of the Arrow API, which you will need to include yourself (for example ``arrow/api.h``). When building C extensions that use the Arrow C++ libraries, you must add -appropriate linker flags. We have provided functions ``pyarrow.get_libraries`` -and ``pyarrow.get_library_dirs`` which return a list of library names and +appropriate linker flags. We have provided functions ``pa.get_libraries`` +and ``pa.get_library_dirs`` which return a list of library names and likely library install locations (if you installed pyarrow with pip or conda). These must be included when declaring your C extensions with setuptools (see below). +.. note:: + + The PyArrow-specific C++ code is now a part of the PyArrow source tree + and not Arrow C++. That means the header files and ``arrow_python`` library + are not necessarily installed in the same location as that of Arrow C++ and + will no longer be automatically findable by CMake. + Initializing the API ~~~~~~~~~~~~~~~~~~~~ diff --git a/docs/source/python/plasma.rst b/docs/source/python/plasma.rst index 51c7b6eafeeae..c12a0ddbe962c 100644 --- a/docs/source/python/plasma.rst +++ b/docs/source/python/plasma.rst @@ -21,6 +21,10 @@ The Plasma In-Memory Object Store ================================= +.. warning:: + + Plasma is deprecated since Arrow 10.0.0. It will be removed in 12.0.0 or so. + .. note:: As present, Plasma is only supported for use on Linux and macOS. diff --git a/docs/source/python/py_arch_overview.svg b/docs/source/python/py_arch_overview.svg deleted file mode 100644 index 11f09eb8c6dda..0000000000000 --- a/docs/source/python/py_arch_overview.svg +++ /dev/null @@ -1 +0,0 @@ - \ No newline at end of file diff --git a/format/adbc.h b/format/adbc.h new file mode 100644 index 0000000000000..a1ff53441db28 --- /dev/null +++ b/format/adbc.h @@ -0,0 +1,1207 @@ +// 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. + +/// \file adbc.h ADBC: Arrow Database connectivity +/// +/// An Arrow-based interface between applications and database +/// drivers. ADBC aims to provide a vendor-independent API for SQL +/// and Substrait-based database access that is targeted at +/// analytics/OLAP use cases. +/// +/// This API is intended to be implemented directly by drivers and +/// used directly by client applications. To assist portability +/// between different vendors, a "driver manager" library is also +/// provided, which implements this same API, but dynamically loads +/// drivers internally and forwards calls appropriately. +/// +/// ADBC uses structs with free functions that operate on those +/// structs to model objects. +/// +/// In general, objects allow serialized access from multiple threads, +/// but not concurrent access. Specific implementations may permit +/// multiple threads. +/// +/// \version 1.0.0 + +#pragma once + +#include +#include + +/// \defgroup Arrow C Data Interface +/// Definitions for the C Data Interface/C Stream Interface. +/// +/// See https://arrow.apache.org/docs/format/CDataInterface.html +/// +/// @{ + +//! @cond Doxygen_Suppress + +#ifdef __cplusplus +extern "C" { +#endif + +// Extra guard for versions of Arrow without the canonical guard +#ifndef ARROW_FLAG_DICTIONARY_ORDERED + +#ifndef ARROW_C_DATA_INTERFACE +#define ARROW_C_DATA_INTERFACE + +#define ARROW_FLAG_DICTIONARY_ORDERED 1 +#define ARROW_FLAG_NULLABLE 2 +#define ARROW_FLAG_MAP_KEYS_SORTED 4 + +struct ArrowSchema { + // Array type description + const char* format; + const char* name; + const char* metadata; + int64_t flags; + int64_t n_children; + struct ArrowSchema** children; + struct ArrowSchema* dictionary; + + // Release callback + void (*release)(struct ArrowSchema*); + // Opaque producer-specific data + void* private_data; +}; + +struct ArrowArray { + // Array data description + int64_t length; + int64_t null_count; + int64_t offset; + int64_t n_buffers; + int64_t n_children; + const void** buffers; + struct ArrowArray** children; + struct ArrowArray* dictionary; + + // Release callback + void (*release)(struct ArrowArray*); + // Opaque producer-specific data + void* private_data; +}; + +#endif // ARROW_C_DATA_INTERFACE + +#ifndef ARROW_C_STREAM_INTERFACE +#define ARROW_C_STREAM_INTERFACE + +struct ArrowArrayStream { + // Callback to get the stream type + // (will be the same for all arrays in the stream). + // + // Return value: 0 if successful, an `errno`-compatible error code otherwise. + // + // If successful, the ArrowSchema must be released independently from the stream. + int (*get_schema)(struct ArrowArrayStream*, struct ArrowSchema* out); + + // Callback to get the next array + // (if no error and the array is released, the stream has ended) + // + // Return value: 0 if successful, an `errno`-compatible error code otherwise. + // + // If successful, the ArrowArray must be released independently from the stream. + int (*get_next)(struct ArrowArrayStream*, struct ArrowArray* out); + + // Callback to get optional detailed error information. + // This must only be called if the last stream operation failed + // with a non-0 return code. + // + // Return value: pointer to a null-terminated character array describing + // the last error, or NULL if no description is available. + // + // The returned pointer is only valid until the next operation on this stream + // (including release). + const char* (*get_last_error)(struct ArrowArrayStream*); + + // Release callback: release the stream's own resources. + // Note that arrays returned by `get_next` must be individually released. + void (*release)(struct ArrowArrayStream*); + + // Opaque producer-specific data + void* private_data; +}; + +#endif // ARROW_C_STREAM_INTERFACE +#endif // ARROW_FLAG_DICTIONARY_ORDERED + +//! @endcond + +/// @} + +#ifndef ADBC +#define ADBC + +// Storage class macros for Windows +// Allow overriding/aliasing with application-defined macros +#if !defined(ADBC_EXPORT) +#if defined(_WIN32) +#if defined(ADBC_EXPORTING) +#define ADBC_EXPORT __declspec(dllexport) +#else +#define ADBC_EXPORT __declspec(dllimport) +#endif // defined(ADBC_EXPORTING) +#else +#define ADBC_EXPORT +#endif // defined(_WIN32) +#endif // !defined(ADBC_EXPORT) + +/// \defgroup adbc-error-handling Error Handling +/// ADBC uses integer error codes to signal errors. To provide more +/// detail about errors, functions may also return an AdbcError via an +/// optional out parameter, which can be inspected. If provided, it is +/// the responsibility of the caller to zero-initialize the AdbcError +/// value. +/// +/// @{ + +/// \brief Error codes for operations that may fail. +typedef uint8_t AdbcStatusCode; + +/// \brief No error. +#define ADBC_STATUS_OK 0 +/// \brief An unknown error occurred. +/// +/// May indicate a driver-side or database-side error. +#define ADBC_STATUS_UNKNOWN 1 +/// \brief The operation is not implemented or supported. +/// +/// May indicate a driver-side or database-side error. +#define ADBC_STATUS_NOT_IMPLEMENTED 2 +/// \brief A requested resource was not found. +/// +/// May indicate a driver-side or database-side error. +#define ADBC_STATUS_NOT_FOUND 3 +/// \brief A requested resource already exists. +/// +/// May indicate a driver-side or database-side error. +#define ADBC_STATUS_ALREADY_EXISTS 4 +/// \brief The arguments are invalid, likely a programming error. +/// +/// For instance, they may be of the wrong format, or out of range. +/// +/// May indicate a driver-side or database-side error. +#define ADBC_STATUS_INVALID_ARGUMENT 5 +/// \brief The preconditions for the operation are not met, likely a +/// programming error. +/// +/// For instance, the object may be uninitialized, or may have not +/// been fully configured. +/// +/// May indicate a driver-side or database-side error. +#define ADBC_STATUS_INVALID_STATE 6 +/// \brief Invalid data was processed (not a programming error). +/// +/// For instance, a division by zero may have occurred during query +/// execution. +/// +/// May indicate a database-side error only. +#define ADBC_STATUS_INVALID_DATA 7 +/// \brief The database's integrity was affected. +/// +/// For instance, a foreign key check may have failed, or a uniqueness +/// constraint may have been violated. +/// +/// May indicate a database-side error only. +#define ADBC_STATUS_INTEGRITY 8 +/// \brief An error internal to the driver or database occurred. +/// +/// May indicate a driver-side or database-side error. +#define ADBC_STATUS_INTERNAL 9 +/// \brief An I/O error occurred. +/// +/// For instance, a remote service may be unavailable. +/// +/// May indicate a driver-side or database-side error. +#define ADBC_STATUS_IO 10 +/// \brief The operation was cancelled, not due to a timeout. +/// +/// May indicate a driver-side or database-side error. +#define ADBC_STATUS_CANCELLED 11 +/// \brief The operation was cancelled due to a timeout. +/// +/// May indicate a driver-side or database-side error. +#define ADBC_STATUS_TIMEOUT 12 +/// \brief Authentication failed. +/// +/// May indicate a database-side error only. +#define ADBC_STATUS_UNAUTHENTICATED 13 +/// \brief The client is not authorized to perform the given operation. +/// +/// May indicate a database-side error only. +#define ADBC_STATUS_UNAUTHORIZED 14 + +/// \brief A detailed error message for an operation. +struct ADBC_EXPORT AdbcError { + /// \brief The error message. + char* message; + + /// \brief A vendor-specific error code, if applicable. + int32_t vendor_code; + + /// \brief A SQLSTATE error code, if provided, as defined by the + /// SQL:2003 standard. If not set, it should be set to + /// "\0\0\0\0\0". + char sqlstate[5]; + + /// \brief Release the contained error. + /// + /// Unlike other structures, this is an embedded callback to make it + /// easier for the driver manager and driver to cooperate. + void (*release)(struct AdbcError* error); +}; + +/// @} + +/// \defgroup adbc-constants Constants +/// @{ + +/// \brief ADBC revision 1.0.0. +/// +/// When passed to an AdbcDriverInitFunc(), the driver parameter must +/// point to an AdbcDriver. +#define ADBC_VERSION_1_0_0 1000000 + +/// \brief Canonical option value for enabling an option. +/// +/// For use as the value in SetOption calls. +#define ADBC_OPTION_VALUE_ENABLED "true" +/// \brief Canonical option value for disabling an option. +/// +/// For use as the value in SetOption calls. +#define ADBC_OPTION_VALUE_DISABLED "false" + +/// \brief The database vendor/product name (e.g. the server name). +/// (type: utf8). +/// +/// \see AdbcConnectionGetInfo +#define ADBC_INFO_VENDOR_NAME 0 +/// \brief The database vendor/product version (type: utf8). +/// +/// \see AdbcConnectionGetInfo +#define ADBC_INFO_VENDOR_VERSION 1 +/// \brief The database vendor/product Arrow library version (type: +/// utf8). +/// +/// \see AdbcConnectionGetInfo +#define ADBC_INFO_VENDOR_ARROW_VERSION 2 + +/// \brief The driver name (type: utf8). +/// +/// \see AdbcConnectionGetInfo +#define ADBC_INFO_DRIVER_NAME 100 +/// \brief The driver version (type: utf8). +/// +/// \see AdbcConnectionGetInfo +#define ADBC_INFO_DRIVER_VERSION 101 +/// \brief The driver Arrow library version (type: utf8). +/// +/// \see AdbcConnectionGetInfo +#define ADBC_INFO_DRIVER_ARROW_VERSION 102 + +/// \brief Return metadata on catalogs, schemas, tables, and columns. +/// +/// \see AdbcConnectionGetObjects +#define ADBC_OBJECT_DEPTH_ALL 0 +/// \brief Return metadata on catalogs only. +/// +/// \see AdbcConnectionGetObjects +#define ADBC_OBJECT_DEPTH_CATALOGS 1 +/// \brief Return metadata on catalogs and schemas. +/// +/// \see AdbcConnectionGetObjects +#define ADBC_OBJECT_DEPTH_DB_SCHEMAS 2 +/// \brief Return metadata on catalogs, schemas, and tables. +/// +/// \see AdbcConnectionGetObjects +#define ADBC_OBJECT_DEPTH_TABLES 3 +/// \brief Return metadata on catalogs, schemas, tables, and columns. +/// +/// \see AdbcConnectionGetObjects +#define ADBC_OBJECT_DEPTH_COLUMNS ADBC_OBJECT_DEPTH_ALL + +/// \brief The name of the canonical option for whether autocommit is +/// enabled. +/// +/// \see AdbcConnectionSetOption +#define ADBC_CONNECTION_OPTION_AUTOCOMMIT "adbc.connection.autocommit" + +/// \brief The name of the canonical option for whether the current +/// connection should be restricted to being read-only. +/// +/// \see AdbcConnectionSetOption +#define ADBC_CONNECTION_OPTION_READ_ONLY "adbc.connection.readonly" + +/// \brief The name of the canonical option for setting the isolation +/// level of a transaction. +/// +/// Should only be used in conjunction with autocommit disabled and +/// AdbcConnectionCommit / AdbcConnectionRollback. If the desired +/// isolation level is not supported by a driver, it should return an +/// appropriate error. +/// +/// \see AdbcConnectionSetOption +#define ADBC_CONNECTION_OPTION_ISOLATION_LEVEL \ + "adbc.connection.transaction.isolation_level" + +/// \brief Use database or driver default isolation level +/// +/// \see AdbcConnectionSetOption +#define ADBC_OPTION_ISOLATION_LEVEL_DEFAULT \ + "adbc.connection.transaction.isolation.default" + +/// \brief The lowest isolation level. Dirty reads are allowed, so one +/// transaction may see not-yet-committed changes made by others. +/// +/// \see AdbcConnectionSetOption +#define ADBC_OPTION_ISOLATION_LEVEL_READ_UNCOMMITTED \ + "adbc.connection.transaction.isolation.read_uncommitted" + +/// \brief Lock-based concurrency control keeps write locks until the +/// end of the transaction, but read locks are released as soon as a +/// SELECT is performed. Non-repeatable reads can occur in this +/// isolation level. +/// +/// More simply put, Read Committed is an isolation level that guarantees +/// that any data read is committed at the moment it is read. It simply +/// restricts the reader from seeing any intermediate, uncommitted, +/// 'dirty' reads. It makes no promise whatsoever that if the transaction +/// re-issues the read, it will find the same data; data is free to change +/// after it is read. +/// +/// \see AdbcConnectionSetOption +#define ADBC_OPTION_ISOLATION_LEVEL_READ_COMMITTED \ + "adbc.connection.transaction.isolation.read_committed" + +/// \brief Lock-based concurrency control keeps read AND write locks +/// (acquired on selection data) until the end of the transaction. +/// +/// However, range-locks are not managed, so phantom reads can occur. +/// Write skew is possible at this isolation level in some systems. +/// +/// \see AdbcConnectionSetOption +#define ADBC_OPTION_ISOLATION_LEVEL_REPEATABLE_READ \ + "adbc.connection.transaction.isolation.repeatable_read" + +/// \brief This isolation guarantees that all reads in the transaction +/// will see a consistent snapshot of the database and the transaction +/// should only successfully commit if no updates conflict with any +/// concurrent updates made since that snapshot. +/// +/// \see AdbcConnectionSetOption +#define ADBC_OPTION_ISOLATION_LEVEL_SNAPSHOT \ + "adbc.connection.transaction.isolation.snapshot" + +/// \brief Serializability requires read and write locks to be released +/// only at the end of the transaction. This includes acquiring range- +/// locks when a select query uses a ranged WHERE clause to avoid +/// phantom reads. +/// +/// \see AdbcConnectionSetOption +#define ADBC_OPTION_ISOLATION_LEVEL_SERIALIZABLE \ + "adbc.connection.transaction.isolation.serializable" + +/// \brief The central distinction between serializability and linearizability +/// is that serializability is a global property; a property of an entire +/// history of operations and transactions. Linearizability is a local +/// property; a property of a single operation/transaction. +/// +/// Linearizability can be viewed as a special case of strict serializability +/// where transactions are restricted to consist of a single operation applied +/// to a single object. +/// +/// \see AdbcConnectionSetOption +#define ADBC_OPTION_ISOLATION_LEVEL_LINEARIZABLE \ + "adbc.connection.transaction.isolation.linearizable" + +/// \defgroup adbc-statement-ingestion Bulk Data Ingestion +/// While it is possible to insert data via prepared statements, it can +/// be more efficient to explicitly perform a bulk insert. For +/// compatible drivers, this can be accomplished by setting up and +/// executing a statement. Instead of setting a SQL query or Substrait +/// plan, bind the source data via AdbcStatementBind, and set the name +/// of the table to be created via AdbcStatementSetOption and the +/// options below. Then, call AdbcStatementExecute with +/// ADBC_OUTPUT_TYPE_UPDATE. +/// +/// @{ + +/// \brief The name of the target table for a bulk insert. +/// +/// The driver should attempt to create the table if it does not +/// exist. If the table exists but has a different schema, +/// ADBC_STATUS_ALREADY_EXISTS should be raised. Else, data should be +/// appended to the target table. +#define ADBC_INGEST_OPTION_TARGET_TABLE "adbc.ingest.target_table" +/// \brief Whether to create (the default) or append. +#define ADBC_INGEST_OPTION_MODE "adbc.ingest.mode" +/// \brief Create the table and insert data; error if the table exists. +#define ADBC_INGEST_OPTION_MODE_CREATE "adbc.ingest.mode.create" +/// \brief Do not create the table, and insert data; error if the +/// table does not exist (ADBC_STATUS_NOT_FOUND) or does not match +/// the schema of the data to append (ADBC_STATUS_ALREADY_EXISTS). +#define ADBC_INGEST_OPTION_MODE_APPEND "adbc.ingest.mode.append" + +/// @} + +/// @} + +/// \defgroup adbc-database Database Initialization +/// Clients first initialize a database, then create a connection +/// (below). This gives the implementation a place to initialize and +/// own any common connection state. For example, in-memory databases +/// can place ownership of the actual database in this object. +/// @{ + +/// \brief An instance of a database. +/// +/// Must be kept alive as long as any connections exist. +struct ADBC_EXPORT AdbcDatabase { + /// \brief Opaque implementation-defined state. + /// This field is NULLPTR iff the connection is unintialized/freed. + void* private_data; + /// \brief The associated driver (used by the driver manager to help + /// track state). + struct AdbcDriver* private_driver; +}; + +/// @} + +/// \defgroup adbc-connection Connection Establishment +/// Functions for creating, using, and releasing database connections. +/// @{ + +/// \brief An active database connection. +/// +/// Provides methods for query execution, managing prepared +/// statements, using transactions, and so on. +/// +/// Connections are not required to be thread-safe, but they can be +/// used from multiple threads so long as clients take care to +/// serialize accesses to a connection. +struct ADBC_EXPORT AdbcConnection { + /// \brief Opaque implementation-defined state. + /// This field is NULLPTR iff the connection is unintialized/freed. + void* private_data; + /// \brief The associated driver (used by the driver manager to help + /// track state). + struct AdbcDriver* private_driver; +}; + +/// @} + +/// \defgroup adbc-statement Managing Statements +/// Applications should first initialize a statement with +/// AdbcStatementNew. Then, the statement should be configured with +/// functions like AdbcStatementSetSqlQuery and +/// AdbcStatementSetOption. Finally, the statement can be executed +/// with AdbcStatementExecuteQuery (or call AdbcStatementPrepare first +/// to turn it into a prepared statement instead). +/// @{ + +/// \brief A container for all state needed to execute a database +/// query, such as the query itself, parameters for prepared +/// statements, driver parameters, etc. +/// +/// Statements may represent queries or prepared statements. +/// +/// Statements may be used multiple times and can be reconfigured +/// (e.g. they can be reused to execute multiple different queries). +/// However, executing a statement (and changing certain other state) +/// will invalidate result sets obtained prior to that execution. +/// +/// Multiple statements may be created from a single connection. +/// However, the driver may block or error if they are used +/// concurrently (whether from a single thread or multiple threads). +/// +/// Statements are not required to be thread-safe, but they can be +/// used from multiple threads so long as clients take care to +/// serialize accesses to a statement. +struct ADBC_EXPORT AdbcStatement { + /// \brief Opaque implementation-defined state. + /// This field is NULLPTR iff the connection is unintialized/freed. + void* private_data; + + /// \brief The associated driver (used by the driver manager to help + /// track state). + struct AdbcDriver* private_driver; +}; + +/// \defgroup adbc-statement-partition Partitioned Results +/// Some backends may internally partition the results. These +/// partitions are exposed to clients who may wish to integrate them +/// with a threaded or distributed execution model, where partitions +/// can be divided among threads or machines and fetched in parallel. +/// +/// To use partitioning, execute the statement with +/// AdbcStatementExecutePartitions to get the partition descriptors. +/// Call AdbcConnectionReadPartition to turn the individual +/// descriptors into ArrowArrayStream instances. This may be done on +/// a different connection than the one the partition was created +/// with, or even in a different process on another machine. +/// +/// Drivers are not required to support partitioning. +/// +/// @{ + +/// \brief The partitions of a distributed/partitioned result set. +struct AdbcPartitions { + /// \brief The number of partitions. + size_t num_partitions; + + /// \brief The partitions of the result set, where each entry (up to + /// num_partitions entries) is an opaque identifier that can be + /// passed to AdbcConnectionReadPartition. + const uint8_t** partitions; + + /// \brief The length of each corresponding entry in partitions. + const size_t* partition_lengths; + + /// \brief Opaque implementation-defined state. + /// This field is NULLPTR iff the connection is unintialized/freed. + void* private_data; + + /// \brief Release the contained partitions. + /// + /// Unlike other structures, this is an embedded callback to make it + /// easier for the driver manager and driver to cooperate. + void (*release)(struct AdbcPartitions* partitions); +}; + +/// @} + +/// @} + +/// \defgroup adbc-driver Driver Initialization +/// +/// These functions are intended to help support integration between a +/// driver and the driver manager. +/// @{ + +/// \brief An instance of an initialized database driver. +/// +/// This provides a common interface for vendor-specific driver +/// initialization routines. Drivers should populate this struct, and +/// applications can call ADBC functions through this struct, without +/// worrying about multiple definitions of the same symbol. +struct ADBC_EXPORT AdbcDriver { + /// \brief Opaque driver-defined state. + /// This field is NULL if the driver is unintialized/freed (but + /// it need not have a value even if the driver is initialized). + void* private_data; + /// \brief Opaque driver manager-defined state. + /// This field is NULL if the driver is unintialized/freed (but + /// it need not have a value even if the driver is initialized). + void* private_manager; + + /// \brief Release the driver and perform any cleanup. + /// + /// This is an embedded callback to make it easier for the driver + /// manager and driver to cooperate. + AdbcStatusCode (*release)(struct AdbcDriver* driver, struct AdbcError* error); + + AdbcStatusCode (*DatabaseInit)(struct AdbcDatabase*, struct AdbcError*); + AdbcStatusCode (*DatabaseNew)(struct AdbcDatabase*, struct AdbcError*); + AdbcStatusCode (*DatabaseSetOption)(struct AdbcDatabase*, const char*, const char*, + struct AdbcError*); + AdbcStatusCode (*DatabaseRelease)(struct AdbcDatabase*, struct AdbcError*); + + AdbcStatusCode (*ConnectionCommit)(struct AdbcConnection*, struct AdbcError*); + AdbcStatusCode (*ConnectionGetInfo)(struct AdbcConnection*, uint32_t*, size_t, + struct ArrowArrayStream*, struct AdbcError*); + AdbcStatusCode (*ConnectionGetObjects)(struct AdbcConnection*, int, const char*, + const char*, const char*, const char**, + const char*, struct ArrowArrayStream*, + struct AdbcError*); + AdbcStatusCode (*ConnectionGetTableSchema)(struct AdbcConnection*, const char*, + const char*, const char*, + struct ArrowSchema*, struct AdbcError*); + AdbcStatusCode (*ConnectionGetTableTypes)(struct AdbcConnection*, + struct ArrowArrayStream*, struct AdbcError*); + AdbcStatusCode (*ConnectionInit)(struct AdbcConnection*, struct AdbcDatabase*, + struct AdbcError*); + AdbcStatusCode (*ConnectionNew)(struct AdbcConnection*, struct AdbcError*); + AdbcStatusCode (*ConnectionSetOption)(struct AdbcConnection*, const char*, const char*, + struct AdbcError*); + AdbcStatusCode (*ConnectionReadPartition)(struct AdbcConnection*, const uint8_t*, + size_t, struct ArrowArrayStream*, + struct AdbcError*); + AdbcStatusCode (*ConnectionRelease)(struct AdbcConnection*, struct AdbcError*); + AdbcStatusCode (*ConnectionRollback)(struct AdbcConnection*, struct AdbcError*); + + AdbcStatusCode (*StatementBind)(struct AdbcStatement*, struct ArrowArray*, + struct ArrowSchema*, struct AdbcError*); + AdbcStatusCode (*StatementBindStream)(struct AdbcStatement*, struct ArrowArrayStream*, + struct AdbcError*); + AdbcStatusCode (*StatementExecuteQuery)(struct AdbcStatement*, struct ArrowArrayStream*, + int64_t*, struct AdbcError*); + AdbcStatusCode (*StatementExecutePartitions)(struct AdbcStatement*, struct ArrowSchema*, + struct AdbcPartitions*, int64_t*, + struct AdbcError*); + AdbcStatusCode (*StatementGetParameterSchema)(struct AdbcStatement*, + struct ArrowSchema*, struct AdbcError*); + AdbcStatusCode (*StatementNew)(struct AdbcConnection*, struct AdbcStatement*, + struct AdbcError*); + AdbcStatusCode (*StatementPrepare)(struct AdbcStatement*, struct AdbcError*); + AdbcStatusCode (*StatementRelease)(struct AdbcStatement*, struct AdbcError*); + AdbcStatusCode (*StatementSetOption)(struct AdbcStatement*, const char*, const char*, + struct AdbcError*); + AdbcStatusCode (*StatementSetSqlQuery)(struct AdbcStatement*, const char*, + struct AdbcError*); + AdbcStatusCode (*StatementSetSubstraitPlan)(struct AdbcStatement*, const uint8_t*, + size_t, struct AdbcError*); +}; + +/// @} + +/// \addtogroup adbc-database +/// @{ + +/// \brief Allocate a new (but uninitialized) database. +ADBC_EXPORT +AdbcStatusCode AdbcDatabaseNew(struct AdbcDatabase* database, struct AdbcError* error); + +/// \brief Set a char* option. +/// +/// Options may be set before AdbcDatabaseInit. Some drivers may +/// support setting options after initialization as well. +/// +/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized +ADBC_EXPORT +AdbcStatusCode AdbcDatabaseSetOption(struct AdbcDatabase* database, const char* key, + const char* value, struct AdbcError* error); + +/// \brief Finish setting options and initialize the database. +/// +/// Some drivers may support setting options after initialization +/// as well. +ADBC_EXPORT +AdbcStatusCode AdbcDatabaseInit(struct AdbcDatabase* database, struct AdbcError* error); + +/// \brief Destroy this database. No connections may exist. +/// \param[in] database The database to release. +/// \param[out] error An optional location to return an error +/// message if necessary. +ADBC_EXPORT +AdbcStatusCode AdbcDatabaseRelease(struct AdbcDatabase* database, + struct AdbcError* error); + +/// @} + +/// \addtogroup adbc-connection +/// @{ + +/// \brief Allocate a new (but uninitialized) connection. +ADBC_EXPORT +AdbcStatusCode AdbcConnectionNew(struct AdbcConnection* connection, + struct AdbcError* error); + +/// \brief Set a char* option. +/// +/// Options may be set before AdbcConnectionInit. Some drivers may +/// support setting options after initialization as well. +/// +/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized +ADBC_EXPORT +AdbcStatusCode AdbcConnectionSetOption(struct AdbcConnection* connection, const char* key, + const char* value, struct AdbcError* error); + +/// \brief Finish setting options and initialize the connection. +/// +/// Some drivers may support setting options after initialization +/// as well. +ADBC_EXPORT +AdbcStatusCode AdbcConnectionInit(struct AdbcConnection* connection, + struct AdbcDatabase* database, struct AdbcError* error); + +/// \brief Destroy this connection. +/// +/// \param[in] connection The connection to release. +/// \param[out] error An optional location to return an error +/// message if necessary. +ADBC_EXPORT +AdbcStatusCode AdbcConnectionRelease(struct AdbcConnection* connection, + struct AdbcError* error); + +/// \defgroup adbc-connection-metadata Metadata +/// Functions for retrieving metadata about the database. +/// +/// Generally, these functions return an ArrowArrayStream that can be +/// consumed to get the metadata as Arrow data. The returned metadata +/// has an expected schema given in the function docstring. Schema +/// fields are nullable unless otherwise marked. While no +/// AdbcStatement is used in these functions, the result set may count +/// as an active statement to the driver for the purposes of +/// concurrency management (e.g. if the driver has a limit on +/// concurrent active statements and it must execute a SQL query +/// internally in order to implement the metadata function). +/// +/// Some functions accept "search pattern" arguments, which are +/// strings that can contain the special character "%" to match zero +/// or more characters, or "_" to match exactly one character. (See +/// the documentation of DatabaseMetaData in JDBC or "Pattern Value +/// Arguments" in the ODBC documentation.) Escaping is not currently +/// supported. +/// +/// @{ + +/// \brief Get metadata about the database/driver. +/// +/// The result is an Arrow dataset with the following schema: +/// +/// Field Name | Field Type +/// ----------------------------|------------------------ +/// info_name | uint32 not null +/// info_value | INFO_SCHEMA +/// +/// INFO_SCHEMA is a dense union with members: +/// +/// Field Name (Type Code) | Field Type +/// ----------------------------|------------------------ +/// string_value (0) | utf8 +/// bool_value (1) | bool +/// int64_value (2) | int64 +/// int32_bitmask (3) | int32 +/// string_list (4) | list +/// int32_to_int32_list_map (5) | map> +/// +/// Each metadatum is identified by an integer code. The recognized +/// codes are defined as constants. Codes [0, 10_000) are reserved +/// for ADBC usage. Drivers/vendors will ignore requests for +/// unrecognized codes (the row will be omitted from the result). +/// +/// \param[in] connection The connection to query. +/// \param[in] info_codes A list of metadata codes to fetch, or NULL +/// to fetch all. +/// \param[in] info_codes_length The length of the info_codes +/// parameter. Ignored if info_codes is NULL. +/// \param[out] out The result set. +/// \param[out] error Error details, if an error occurs. +ADBC_EXPORT +AdbcStatusCode AdbcConnectionGetInfo(struct AdbcConnection* connection, + uint32_t* info_codes, size_t info_codes_length, + struct ArrowArrayStream* out, + struct AdbcError* error); + +/// \brief Get a hierarchical view of all catalogs, database schemas, +/// tables, and columns. +/// +/// The result is an Arrow dataset with the following schema: +/// +/// | Field Name | Field Type | +/// |--------------------------|-------------------------| +/// | catalog_name | utf8 | +/// | catalog_db_schemas | list | +/// +/// DB_SCHEMA_SCHEMA is a Struct with fields: +/// +/// | Field Name | Field Type | +/// |--------------------------|-------------------------| +/// | db_schema_name | utf8 | +/// | db_schema_tables | list | +/// +/// TABLE_SCHEMA is a Struct with fields: +/// +/// | Field Name | Field Type | +/// |--------------------------|-------------------------| +/// | table_name | utf8 not null | +/// | table_type | utf8 not null | +/// | table_columns | list | +/// | table_constraints | list | +/// +/// COLUMN_SCHEMA is a Struct with fields: +/// +/// | Field Name | Field Type | Comments | +/// |--------------------------|-------------------------|----------| +/// | column_name | utf8 not null | | +/// | ordinal_position | int32 | (1) | +/// | remarks | utf8 | (2) | +/// | xdbc_data_type | int16 | (3) | +/// | xdbc_type_name | utf8 | (3) | +/// | xdbc_column_size | int32 | (3) | +/// | xdbc_decimal_digits | int16 | (3) | +/// | xdbc_num_prec_radix | int16 | (3) | +/// | xdbc_nullable | int16 | (3) | +/// | xdbc_column_def | utf8 | (3) | +/// | xdbc_sql_data_type | int16 | (3) | +/// | xdbc_datetime_sub | int16 | (3) | +/// | xdbc_char_octet_length | int32 | (3) | +/// | xdbc_is_nullable | utf8 | (3) | +/// | xdbc_scope_catalog | utf8 | (3) | +/// | xdbc_scope_schema | utf8 | (3) | +/// | xdbc_scope_table | utf8 | (3) | +/// | xdbc_is_autoincrement | bool | (3) | +/// | xdbc_is_generatedcolumn | bool | (3) | +/// +/// 1. The column's ordinal position in the table (starting from 1). +/// 2. Database-specific description of the column. +/// 3. Optional value. Should be null if not supported by the driver. +/// xdbc_ values are meant to provide JDBC/ODBC-compatible metadata +/// in an agnostic manner. +/// +/// CONSTRAINT_SCHEMA is a Struct with fields: +/// +/// | Field Name | Field Type | Comments | +/// |--------------------------|-------------------------|----------| +/// | constraint_name | utf8 | | +/// | constraint_type | utf8 not null | (1) | +/// | constraint_column_names | list not null | (2) | +/// | constraint_column_usage | list | (3) | +/// +/// 1. One of 'CHECK', 'FOREIGN KEY', 'PRIMARY KEY', or 'UNIQUE'. +/// 2. The columns on the current table that are constrained, in +/// order. +/// 3. For FOREIGN KEY only, the referenced table and columns. +/// +/// USAGE_SCHEMA is a Struct with fields: +/// +/// | Field Name | Field Type | +/// |--------------------------|-------------------------| +/// | fk_catalog | utf8 | +/// | fk_db_schema | utf8 | +/// | fk_table | utf8 not null | +/// | fk_column_name | utf8 not null | +/// +/// \param[in] connection The database connection. +/// \param[in] depth The level of nesting to display. If 0, display +/// all levels. If 1, display only catalogs (i.e. catalog_schemas +/// will be null). If 2, display only catalogs and schemas +/// (i.e. db_schema_tables will be null), and so on. +/// \param[in] catalog Only show tables in the given catalog. If NULL, +/// do not filter by catalog. If an empty string, only show tables +/// without a catalog. May be a search pattern (see section +/// documentation). +/// \param[in] db_schema Only show tables in the given database schema. If +/// NULL, do not filter by database schema. If an empty string, only show +/// tables without a database schema. May be a search pattern (see section +/// documentation). +/// \param[in] table_name Only show tables with the given name. If NULL, do not +/// filter by name. May be a search pattern (see section documentation). +/// \param[in] table_type Only show tables matching one of the given table +/// types. If NULL, show tables of any type. Valid table types can be fetched +/// from GetTableTypes. Terminate the list with a NULL entry. +/// \param[in] column_name Only show columns with the given name. If +/// NULL, do not filter by name. May be a search pattern (see +/// section documentation). +/// \param[out] out The result set. +/// \param[out] error Error details, if an error occurs. +ADBC_EXPORT +AdbcStatusCode AdbcConnectionGetObjects(struct AdbcConnection* connection, int depth, + const char* catalog, const char* db_schema, + const char* table_name, const char** table_type, + const char* column_name, + struct ArrowArrayStream* out, + struct AdbcError* error); + +/// \brief Get the Arrow schema of a table. +/// +/// \param[in] connection The database connection. +/// \param[in] catalog The catalog (or nullptr if not applicable). +/// \param[in] db_schema The database schema (or nullptr if not applicable). +/// \param[in] table_name The table name. +/// \param[out] schema The table schema. +/// \param[out] error Error details, if an error occurs. +ADBC_EXPORT +AdbcStatusCode AdbcConnectionGetTableSchema(struct AdbcConnection* connection, + const char* catalog, const char* db_schema, + const char* table_name, + struct ArrowSchema* schema, + struct AdbcError* error); + +/// \brief Get a list of table types in the database. +/// +/// The result is an Arrow dataset with the following schema: +/// +/// Field Name | Field Type +/// ---------------|-------------- +/// table_type | utf8 not null +/// +/// \param[in] connection The database connection. +/// \param[out] out The result set. +/// \param[out] error Error details, if an error occurs. +ADBC_EXPORT +AdbcStatusCode AdbcConnectionGetTableTypes(struct AdbcConnection* connection, + struct ArrowArrayStream* out, + struct AdbcError* error); + +/// @} + +/// \defgroup adbc-connection-partition Partitioned Results +/// Some databases may internally partition the results. These +/// partitions are exposed to clients who may wish to integrate them +/// with a threaded or distributed execution model, where partitions +/// can be divided among threads or machines for processing. +/// +/// Drivers are not required to support partitioning. +/// +/// Partitions are not ordered. If the result set is sorted, +/// implementations should return a single partition. +/// +/// @{ + +/// \brief Construct a statement for a partition of a query. The +/// results can then be read independently. +/// +/// A partition can be retrieved from AdbcPartitions. +/// +/// \param[in] connection The connection to use. This does not have +/// to be the same connection that the partition was created on. +/// \param[in] serialized_partition The partition descriptor. +/// \param[in] serialized_length The partition descriptor length. +/// \param[out] out The result set. +/// \param[out] error Error details, if an error occurs. +ADBC_EXPORT +AdbcStatusCode AdbcConnectionReadPartition(struct AdbcConnection* connection, + const uint8_t* serialized_partition, + size_t serialized_length, + struct ArrowArrayStream* out, + struct AdbcError* error); + +/// @} + +/// \defgroup adbc-connection-transaction Transaction Semantics +/// +/// Connections start out in auto-commit mode by default (if +/// applicable for the given vendor). Use AdbcConnectionSetOption and +/// ADBC_CONNECTION_OPTION_AUTO_COMMIT to change this. +/// +/// @{ + +/// \brief Commit any pending transactions. Only used if autocommit is +/// disabled. +/// +/// Behavior is undefined if this is mixed with SQL transaction +/// statements. +ADBC_EXPORT +AdbcStatusCode AdbcConnectionCommit(struct AdbcConnection* connection, + struct AdbcError* error); + +/// \brief Roll back any pending transactions. Only used if autocommit +/// is disabled. +/// +/// Behavior is undefined if this is mixed with SQL transaction +/// statements. +ADBC_EXPORT +AdbcStatusCode AdbcConnectionRollback(struct AdbcConnection* connection, + struct AdbcError* error); + +/// @} + +/// @} + +/// \addtogroup adbc-statement +/// @{ + +/// \brief Create a new statement for a given connection. +/// +/// Set options on the statement, then call AdbcStatementExecuteQuery +/// or AdbcStatementPrepare. +ADBC_EXPORT +AdbcStatusCode AdbcStatementNew(struct AdbcConnection* connection, + struct AdbcStatement* statement, struct AdbcError* error); + +/// \brief Destroy a statement. +/// \param[in] statement The statement to release. +/// \param[out] error An optional location to return an error +/// message if necessary. +ADBC_EXPORT +AdbcStatusCode AdbcStatementRelease(struct AdbcStatement* statement, + struct AdbcError* error); + +/// \brief Execute a statement and get the results. +/// +/// This invalidates any prior result sets. +/// +/// \param[in] statement The statement to execute. +/// \param[out] out The results. Pass NULL if the client does not +/// expect a result set. +/// \param[out] rows_affected The number of rows affected if known, +/// else -1. Pass NULL if the client does not want this information. +/// \param[out] error An optional location to return an error +/// message if necessary. +ADBC_EXPORT +AdbcStatusCode AdbcStatementExecuteQuery(struct AdbcStatement* statement, + struct ArrowArrayStream* out, + int64_t* rows_affected, struct AdbcError* error); + +/// \brief Turn this statement into a prepared statement to be +/// executed multiple times. +/// +/// This invalidates any prior result sets. +ADBC_EXPORT +AdbcStatusCode AdbcStatementPrepare(struct AdbcStatement* statement, + struct AdbcError* error); + +/// \defgroup adbc-statement-sql SQL Semantics +/// Functions for executing SQL queries, or querying SQL-related +/// metadata. Drivers are not required to support both SQL and +/// Substrait semantics. If they do, it may be via converting +/// between representations internally. +/// @{ + +/// \brief Set the SQL query to execute. +/// +/// The query can then be executed with AdbcStatementExecute. For +/// queries expected to be executed repeatedly, AdbcStatementPrepare +/// the statement first. +/// +/// \param[in] statement The statement. +/// \param[in] query The query to execute. +/// \param[out] error Error details, if an error occurs. +ADBC_EXPORT +AdbcStatusCode AdbcStatementSetSqlQuery(struct AdbcStatement* statement, + const char* query, struct AdbcError* error); + +/// @} + +/// \defgroup adbc-statement-substrait Substrait Semantics +/// Functions for executing Substrait plans, or querying +/// Substrait-related metadata. Drivers are not required to support +/// both SQL and Substrait semantics. If they do, it may be via +/// converting between representations internally. +/// @{ + +/// \brief Set the Substrait plan to execute. +/// +/// The query can then be executed with AdbcStatementExecute. For +/// queries expected to be executed repeatedly, AdbcStatementPrepare +/// the statement first. +/// +/// \param[in] statement The statement. +/// \param[in] plan The serialized substrait.Plan to execute. +/// \param[in] length The length of the serialized plan. +/// \param[out] error Error details, if an error occurs. +ADBC_EXPORT +AdbcStatusCode AdbcStatementSetSubstraitPlan(struct AdbcStatement* statement, + const uint8_t* plan, size_t length, + struct AdbcError* error); + +/// @} + +/// \brief Bind Arrow data. This can be used for bulk inserts or +/// prepared statements. +/// +/// \param[in] statement The statement to bind to. +/// \param[in] values The values to bind. The driver will call the +/// release callback itself, although it may not do this until the +/// statement is released. +/// \param[in] schema The schema of the values to bind. +/// \param[out] error An optional location to return an error message +/// if necessary. +ADBC_EXPORT +AdbcStatusCode AdbcStatementBind(struct AdbcStatement* statement, + struct ArrowArray* values, struct ArrowSchema* schema, + struct AdbcError* error); + +/// \brief Bind Arrow data. This can be used for bulk inserts or +/// prepared statements. +/// \param[in] statement The statement to bind to. +/// \param[in] stream The values to bind. The driver will call the +/// release callback itself, although it may not do this until the +/// statement is released. +/// \param[out] error An optional location to return an error message +/// if necessary. +ADBC_EXPORT +AdbcStatusCode AdbcStatementBindStream(struct AdbcStatement* statement, + struct ArrowArrayStream* stream, + struct AdbcError* error); + +/// \brief Get the schema for bound parameters. +/// +/// This retrieves an Arrow schema describing the number, names, and +/// types of the parameters in a parameterized statement. The fields +/// of the schema should be in order of the ordinal position of the +/// parameters; named parameters should appear only once. +/// +/// If the parameter does not have a name, or the name cannot be +/// determined, the name of the corresponding field in the schema will +/// be an empty string. If the type cannot be determined, the type of +/// the corresponding field will be NA (NullType). +/// +/// This should be called after AdbcStatementPrepare. +/// +/// \return ADBC_STATUS_NOT_IMPLEMENTED if the schema cannot be determined. +ADBC_EXPORT +AdbcStatusCode AdbcStatementGetParameterSchema(struct AdbcStatement* statement, + struct ArrowSchema* schema, + struct AdbcError* error); + +/// \brief Set a string option on a statement. +ADBC_EXPORT +AdbcStatusCode AdbcStatementSetOption(struct AdbcStatement* statement, const char* key, + const char* value, struct AdbcError* error); + +/// \addtogroup adbc-statement-partition +/// @{ + +/// \brief Execute a statement and get the results as a partitioned +/// result set. +/// +/// \param[in] statement The statement to execute. +/// \param[out] schema The schema of the result set. +/// \param[out] partitions The result partitions. +/// \param[out] rows_affected The number of rows affected if known, +/// else -1. Pass NULL if the client does not want this information. +/// \param[out] error An optional location to return an error +/// message if necessary. +/// \return ADBC_STATUS_NOT_IMPLEMENTED if the driver does not support +/// partitioned results +ADBC_EXPORT +AdbcStatusCode AdbcStatementExecutePartitions(struct AdbcStatement* statement, + struct ArrowSchema* schema, + struct AdbcPartitions* partitions, + int64_t* rows_affected, + struct AdbcError* error); + +/// @} + +/// @} + +/// \addtogroup adbc-driver +/// @{ + +/// \brief Common entry point for drivers via the driver manager +/// (which uses dlopen(3)/LoadLibrary). The driver manager is told +/// to load a library and call a function of this type to load the +/// driver. +/// +/// Although drivers may choose any name for this function, the +/// recommended name is "AdbcDriverInit". +/// +/// \param[in] version The ADBC revision to attempt to initialize (see +/// ADBC_VERSION_1_0_0). +/// \param[out] driver The table of function pointers to +/// initialize. Should be a pointer to the appropriate struct for +/// the given version (see the documentation for the version). +/// \param[out] error An optional location to return an error message +/// if necessary. +/// \return ADBC_STATUS_OK if the driver was initialized, or +/// ADBC_STATUS_NOT_IMPLEMENTED if the version is not supported. In +/// that case, clients may retry with a different version. +typedef AdbcStatusCode (*AdbcDriverInitFunc)(int version, void* driver, + struct AdbcError* error); + +/// @} + +#endif // ADBC + +#ifdef __cplusplus +} +#endif diff --git a/go.work b/go.work index ccfe97d5510a4..104e3bda17ba0 100644 --- a/go.work +++ b/go.work @@ -14,10 +14,11 @@ // See the License for the specific language governing permissions and // limitations under the License. - go 1.18 use ( ./go ./go/arrow/compute ) + +replace github.com/apache/arrow/go/v10 v10.0.0 => ./go diff --git a/go.work.sum b/go.work.sum index 5704d0dab2080..21a771896242b 100644 --- a/go.work.sum +++ b/go.work.sum @@ -1,44 +1 @@ -github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= -github.com/goccy/go-json v0.9.11 h1:/pAaQDLHEoCq/5FFmSKBswWmK6H0e8g4159Kc/X/nqk= -github.com/goccy/go-json v0.9.11/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= -github.com/google/flatbuffers v2.0.8+incompatible h1:ivUb1cGomAB101ZM1T0nOiWz9pSrTMoa9+EiY7igmkM= -github.com/google/flatbuffers v2.0.8+incompatible/go.mod h1:1AeVuKshWv4vARoZatz6mlQ0JxURH0Kv5+zNeJKJCa8= -github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= -github.com/kr/pretty v0.3.0 h1:WgNl7dwNpEZ6jJ9k1snq4pZsg7DOEN8hP9Xw0Tsjwk0= -github.com/kr/pretty v0.3.0/go.mod h1:640gp4NfQd8pI5XOwp5fnNeVWj67G7CFk/SaSQn7NBk= -github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= -github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/mattn/go-isatty v0.0.16 h1:bq3VjFmv/sOjHtdEhmkEV4x1AJtvUvOJ2PFAZ5+peKQ= -github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= -github.com/mattn/go-sqlite3 v1.14.14/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU= -github.com/rogpeppe/go-internal v1.6.1 h1:/FiVV8dS/e+YqF2JvO3yXRFbBLTIuSDkuC7aBOAvL+k= -github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= -golang.org/x/exp v0.0.0-20220827204233-334a2380cb91 h1:tnebWN09GYg9OLPss1KXj8txwZc6X6uMr6VFdcGNbHw= -golang.org/x/exp v0.0.0-20220827204233-334a2380cb91/go.mod h1:cyybsKvd6eL0RnXn6p/Grxp8F5bW7iYuBgsNCOHpMYE= -golang.org/x/sync v0.0.0-20220819030929-7fc1605a5dde h1:ejfdSekXMDxDLbRrJMwUk6KnSLZ2McaUCVcIKM+N6jc= -golang.org/x/sync v0.0.0-20220819030929-7fc1605a5dde/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220829200755-d48e67d00261 h1:v6hYoSR9T5oet+pMXwUWkbiVqx/63mlHjefrHmxwfeY= -golang.org/x/sys v0.0.0-20220829200755-d48e67d00261/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -google.golang.org/grpc v1.49.0 h1:WTLtQzmQori5FUH25Pq4WT22oCsv8USpQ+F6rqtsmxw= -google.golang.org/grpc v1.49.0/go.mod h1:ZgQEeidpAuNRZ8iRrlBKXZQP1ghovWIVhdJRyCDK+GI= -gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= -gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= -gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= -modernc.org/cc/v3 v3.36.2/go.mod h1:NFUHyPn4ekoC/JHeZFfZurN6ixxawE1BnVonP/oahEI= -modernc.org/cc/v3 v3.36.3 h1:uISP3F66UlixxWEcKuIWERa4TwrZENHSL8tWxZz8bHg= -modernc.org/cc/v3 v3.36.3/go.mod h1:NFUHyPn4ekoC/JHeZFfZurN6ixxawE1BnVonP/oahEI= -modernc.org/ccgo/v3 v3.16.9 h1:AXquSwg7GuMk11pIdw7fmO1Y/ybgazVkMhsZWCV0mHM= -modernc.org/ccgo/v3 v3.16.9/go.mod h1:zNMzC9A9xeNUepy6KuZBbugn3c0Mc9TeiJO4lgvkJDo= -modernc.org/libc v1.17.0/go.mod h1:XsgLldpP4aWlPlsjqKRdHPqCxCjISdHfM/yeWC5GyW0= -modernc.org/libc v1.17.1 h1:Q8/Cpi36V/QBfuQaFVeisEBs3WqoGAJprZzmf7TfEYI= -modernc.org/libc v1.17.1/go.mod h1:FZ23b+8LjxZs7XtFMbSzL/EhPxNbfZbErxEHc7cbD9s= -modernc.org/mathutil v1.5.0 h1:rV0Ko/6SfM+8G+yKiyI830l3Wuz1zRutdslNoQ0kfiQ= -modernc.org/mathutil v1.5.0/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E= -modernc.org/memory v1.2.0/go.mod h1:/0wo5ibyrQiaoUoH7f9D8dnglAmILJ5/cxZlRECf+Nw= -modernc.org/memory v1.2.1 h1:dkRh86wgmq/bJu2cAS2oqBCz/KsMZU7TUM4CibQ7eBs= -modernc.org/memory v1.2.1/go.mod h1:PkUhL0Mugw21sHPeskwZW4D6VscE/GQJOnIpCnW6pSU= -modernc.org/sqlite v1.18.1 h1:ko32eKt3jf7eqIkCgPAeHMBXw3riNSLhl2f3loEF7o8= -modernc.org/sqlite v1.18.1/go.mod h1:6ho+Gow7oX5V+OiOQ6Tr4xeqbx13UZ6t+Fw9IRUG4d4= -modernc.org/strutil v1.1.3 h1:fNMm+oJklMGYfU9Ylcywl0CO5O6nTfaowNsh2wpPjzY= -modernc.org/strutil v1.1.3/go.mod h1:MEHNA7PdEnEwLvspRMtWTNnp2nnyvMfkimT1NKNAGbw= +github.com/apache/arrow/go/v10 v10.0.0-20221012182438-704536b48d7d/go.mod h1:CH7QuXsCbYxM4utMqmnie6Xy+4SqMD0yaSdOalg0M/o= diff --git a/go/arrow/compute/doc.go b/go/arrow/compute/doc.go index ee19cd4f965f8..07610f7eb7621 100644 --- a/go/arrow/compute/doc.go +++ b/go/arrow/compute/doc.go @@ -17,6 +17,17 @@ // Package compute is a native-go implementation of an Acero-like // arrow compute engine. // +// For now in order to properly use the compute library you'll need to run: +// +// go mod edit -replace github.com/apache/arrow/go/v${major_version}/arrow/compute=github.com/apache/arrow/go/arrow/compute +// +// In order to import "github.com/apache/arrow/go/v${major_version}/arrow/compute" +// in your package. This is due to it being a separate module so that it can +// utilize go1.18. After the release of go1.20, the Arrow modules will be bumped to +// a minimum go.mod of go1.18 and the compute package will be integrated +// into the arrow module proper, rather than being a separate module. At that +// point, the replace directive will no longer be needed. +// // While consumers of Arrow that are able to use CGO could utilize the // C Data API (using the cdata package) and could link against the // acero library directly, there are consumers who cannot use CGO. This diff --git a/go/arrow/compute/go.mod b/go/arrow/compute/go.mod index 9aa0379f6891b..9c69ac278a247 100644 --- a/go/arrow/compute/go.mod +++ b/go/arrow/compute/go.mod @@ -18,10 +18,8 @@ module github.com/apache/arrow/go/v10/arrow/compute go 1.18 -replace github.com/apache/arrow/go/v10 => ../../ - require ( - github.com/apache/arrow/go/v10 v10.0.0-00010101000000-000000000000 + github.com/apache/arrow/go/v10 v10.0.0 github.com/stretchr/testify v1.8.0 golang.org/x/exp v0.0.0-20220827204233-334a2380cb91 golang.org/x/sync v0.0.0-20220819030929-7fc1605a5dde diff --git a/go/arrow/compute/go.sum b/go/arrow/compute/go.sum index cc95f335c211c..9c12e5c3b484e 100644 --- a/go/arrow/compute/go.sum +++ b/go/arrow/compute/go.sum @@ -1,6 +1,8 @@ github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c h1:RGWPOewvKIROun94nF7v2cua9qP+thov/7M50KEoeSU= github.com/andybalholm/brotli v1.0.4 h1:V7DdXeJtZscaqfNuAdSRuRFzuiKlHSC/Zh3zl9qY3JY= github.com/andybalholm/brotli v1.0.4/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= +github.com/apache/arrow/go/v10 v10.0.0-20221012182438-704536b48d7d h1:+9f5QDC7t2w4VsXcT2aGs1gg5vYzPnZIhmY4Mkf6oUk= +github.com/apache/arrow/go/v10 v10.0.0-20221012182438-704536b48d7d/go.mod h1:CH7QuXsCbYxM4utMqmnie6Xy+4SqMD0yaSdOalg0M/o= github.com/apache/thrift v0.16.0 h1:qEy6UW60iVOlUy+b9ZR0d5WzUWYGOo4HfopoyBaNmoY= github.com/apache/thrift v0.16.0/go.mod h1:PHK3hniurgQaNMZYaCLEqXKsYK8upmhPbmdP2FXSqgU= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= diff --git a/go/go.mod b/go/go.mod index 25ca1e084c75d..26178257e1c6a 100644 --- a/go/go.mod +++ b/go/go.mod @@ -54,6 +54,7 @@ require ( github.com/mattn/go-isatty v0.0.16 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect + github.com/rogpeppe/go-internal v1.9.0 // indirect github.com/stretchr/objx v0.4.0 // indirect golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4 // indirect golang.org/x/net v0.0.0-20220722155237-a158d28d115b // indirect diff --git a/go/go.sum b/go/go.sum index e658e9114efb7..04695d5559461 100644 --- a/go/go.sum +++ b/go/go.sum @@ -126,6 +126,7 @@ github.com/phpdave11/gofpdi v1.0.12/go.mod h1:vBmVV0Do6hSBHC8uKUQ71JGW+ZGQq74llk github.com/phpdave11/gofpdi v1.0.13/go.mod h1:vBmVV0Do6hSBHC8uKUQ71JGW+ZGQq74llk/7bXwjDoI= github.com/pierrec/lz4/v4 v4.1.15 h1:MO0/ucJhngq7299dKLwIMtgTfbkoSPF6AoMYDd8Q4q0= github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= +github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= @@ -134,8 +135,8 @@ github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1: github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 h1:OdAsTTz6OkFY5QxjkYwrChwuRruF69c169dPK26NUlk= github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= -github.com/rogpeppe/go-internal v1.6.1 h1:/FiVV8dS/e+YqF2JvO3yXRFbBLTIuSDkuC7aBOAvL+k= github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= +github.com/rogpeppe/go-internal v1.9.0 h1:73kH8U+JUqXU8lRuOHeVHaa/SZPifC7BkcraZVejAe8= github.com/ruudk/golang-pdf417 v0.0.0-20181029194003-1af4ab5afa58/go.mod h1:6lfFZQK844Gfx8o5WFuvpxWRwnSoipWe/p622j1v06w= github.com/ruudk/golang-pdf417 v0.0.0-20201230142125-a7e3863a1245/go.mod h1:pQAZKsJ8yyVxGRWYNEm9oFB8ieLgKFnamEyDmSA0BRk= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= @@ -149,7 +150,6 @@ github.com/stretchr/testify v1.8.0 h1:pSgiaMZlXftHpm5L7V1+rVB+AZJydKsMxsQBIJw4PK github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.4.1/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -github.com/yuin/goldmark v1.4.13 h1:fVcFKWvrslecOb/tg+Cc05dkeYx540o0FuFt3nUVDoE= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= github.com/zeebo/assert v1.3.0 h1:g7C04CbJuIDKNPFHmsk4hwZDO5O+kntRxzaUoNXj+IQ= github.com/zeebo/assert v1.3.0/go.mod h1:Pq9JiuJQpG8JLJdtkwrJESF0Foym2/D9XMU5ciN/wJ0= diff --git a/java/adapter/orc/pom.xml b/java/adapter/orc/pom.xml index 9a0c9d95f8812..444b1e6d2b069 100644 --- a/java/adapter/orc/pom.xml +++ b/java/adapter/orc/pom.xml @@ -35,7 +35,7 @@ org.apache.orc orc-core - 1.7.6 + 1.8.0 test diff --git a/java/dataset/src/main/cpp/jni_wrapper.cc b/java/dataset/src/main/cpp/jni_wrapper.cc index 6fffc546eaf2e..aa7d767023242 100644 --- a/java/dataset/src/main/cpp/jni_wrapper.cc +++ b/java/dataset/src/main/cpp/jni_wrapper.cc @@ -94,6 +94,10 @@ arrow::Result> GetFileFormat( #ifdef ARROW_ORC case 2: return std::make_shared(); +#endif +#ifdef ARROW_CSV + case 3: + return std::make_shared(); #endif default: std::string error_message = diff --git a/java/dataset/src/main/java/org/apache/arrow/dataset/file/FileFormat.java b/java/dataset/src/main/java/org/apache/arrow/dataset/file/FileFormat.java index b428b254b10ca..aad4fa5f2af48 100644 --- a/java/dataset/src/main/java/org/apache/arrow/dataset/file/FileFormat.java +++ b/java/dataset/src/main/java/org/apache/arrow/dataset/file/FileFormat.java @@ -24,6 +24,7 @@ public enum FileFormat { PARQUET(0), ARROW_IPC(1), ORC(2), + CSV(3), NONE(-1); private final int id; diff --git a/java/dataset/src/test/java/org/apache/arrow/dataset/CsvWriteSupport.java b/java/dataset/src/test/java/org/apache/arrow/dataset/CsvWriteSupport.java new file mode 100644 index 0000000000000..954408ce25ecc --- /dev/null +++ b/java/dataset/src/test/java/org/apache/arrow/dataset/CsvWriteSupport.java @@ -0,0 +1,49 @@ +/* + * 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. + */ + +package org.apache.arrow.dataset; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Random; + +public class CsvWriteSupport { + private final URI uri; + private final Random random = new Random(); + + public CsvWriteSupport(File outputFolder) throws URISyntaxException { + uri = new URI("file", outputFolder.getPath() + File.separator + "generated-" + random.nextLong() + ".csv", null); + } + + public static CsvWriteSupport writeTempFile(File outputFolder, String... values) + throws URISyntaxException, IOException { + CsvWriteSupport writer = new CsvWriteSupport(outputFolder); + try (FileWriter addValues = new FileWriter(new File(writer.uri), true)) { + for (Object value : values) { + addValues.write(value + "\n"); + } + } + return writer; + } + + public String getOutputURI() { + return uri.toString(); + } +} diff --git a/java/dataset/src/test/java/org/apache/arrow/dataset/file/TestFileSystemDataset.java b/java/dataset/src/test/java/org/apache/arrow/dataset/file/TestFileSystemDataset.java index 9dc5f2b655a83..b8a13937a8aad 100644 --- a/java/dataset/src/test/java/org/apache/arrow/dataset/file/TestFileSystemDataset.java +++ b/java/dataset/src/test/java/org/apache/arrow/dataset/file/TestFileSystemDataset.java @@ -37,6 +37,7 @@ import java.util.concurrent.Executors; import java.util.stream.Collectors; +import org.apache.arrow.dataset.CsvWriteSupport; import org.apache.arrow.dataset.OrcWriteSupport; import org.apache.arrow.dataset.ParquetWriteSupport; import org.apache.arrow.dataset.jni.NativeDataset; @@ -361,6 +362,30 @@ public void testBaseOrcRead() throws Exception { AutoCloseables.close(factory); } + @Test + public void testBaseCsvRead() throws Exception { + CsvWriteSupport writeSupport = CsvWriteSupport.writeTempFile( + TMP.newFolder(), "Name,Language", "Juno,Java", "Peter,Python", "Celin,C++"); + String expectedJsonUnordered = "[[\"Juno\", \"Java\"], [\"Peter\", \"Python\"], [\"Celin\", \"C++\"]]"; + ScanOptions options = new ScanOptions(100); + try ( + FileSystemDatasetFactory factory = new FileSystemDatasetFactory(rootAllocator(), NativeMemoryPool.getDefault(), + FileFormat.CSV, writeSupport.getOutputURI()) + ) { + List datum = collectResultFromFactory(factory, options); + Schema schema = inferResultSchemaFromFactory(factory, options); + + assertScanBatchesProduced(factory, options); + assertEquals(1, datum.size()); + assertEquals(2, schema.getFields().size()); + assertEquals("Name", schema.getFields().get(0).getName()); + + checkParquetReadResult(schema, expectedJsonUnordered, datum); + + AutoCloseables.close(datum); + } + } + private void checkParquetReadResult(Schema schema, String expectedJson, List actual) throws IOException { final ObjectMapper json = new ObjectMapper(); diff --git a/java/pom.xml b/java/pom.xml index 0438bb8fda88c..666700d36497e 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -916,7 +916,7 @@ generate-cdata-dylib_so - java-dist/lib + java-dist @@ -981,8 +981,18 @@ generate-jni-dylib_so - java-dist/lib + java-dist false + ON + ON + ON + OFF + OFF + OFF + ON + OFF + ON + OFF @@ -1015,16 +1025,16 @@ -S cpp -B cpp-jni -DARROW_BUILD_SHARED=OFF - -DARROW_CSV=ON + -DARROW_CSV=${ARROW_CSV} -DARROW_DATASET=ON -DARROW_DEPENDENCY_SOURCE=BUNDLED -DARROW_DEPENDENCY_USE_SHARED=OFF -DARROW_FILESYSTEM=ON - -DARROW_GANDIVA=ON + -DARROW_GANDIVA=${ARROW_GANDIVA} -DARROW_GANDIVA_STATIC_LIBSTDCPP=ON - -DARROW_ORC=ON - -DARROW_PARQUET=ON - -DARROW_PLASMA=ON + -DARROW_ORC=${ARROW_ORC} + -DARROW_PARQUET=${ARROW_PARQUET} + -DARROW_PLASMA=${ARROW_PLASMA} -DARROW_S3=ON -DARROW_USE_CCACHE=ON -DCMAKE_BUILD_TYPE=Release @@ -1062,12 +1072,16 @@ -S java -B java-jni - -DARROW_JAVA_JNI_ENABLE_C=OFF + -DARROW_JAVA_JNI_ENABLE_C=${ARROW_JAVA_JNI_ENABLE_C} + -DARROW_JAVA_JNI_ENABLE_DATASET=${ARROW_JAVA_JNI_ENABLE_DATASET} + -DARROW_JAVA_JNI_ENABLE_GANDIVA=${ARROW_JAVA_JNI_ENABLE_GANDIVA} + -DARROW_JAVA_JNI_ENABLE_ORC=${ARROW_JAVA_JNI_ENABLE_ORC} + -DARROW_JAVA_JNI_ENABLE_PLASMA=${ARROW_JAVA_JNI_ENABLE_PLASMA} -DARROW_JAVA_JNI_ENABLE_DEFAULT=ON -DBUILD_TESTING=OFF -DCMAKE_BUILD_TYPE=Release -DCMAKE_INSTALL_LIBDIR=lib - -DCMAKE_INSTALL_PREFIX=${arrow.c.jni.dist.dir} + -DCMAKE_INSTALL_PREFIX=${arrow.dataset.jni.dist.dir} -DCMAKE_PREFIX_PATH=${project.basedir}/../java-dist ../ diff --git a/python/pyarrow/__init__.py b/python/pyarrow/__init__.py index 19d630bc0abf4..bcb6b30e715f9 100644 --- a/python/pyarrow/__init__.py +++ b/python/pyarrow/__init__.py @@ -341,13 +341,21 @@ def __getattr__(name): def _plasma_store_entry_point(): - """Entry point for starting the plasma store. + """ + DEPRECATED: Entry point for starting the plasma store. This can be used by invoking e.g. ``plasma_store -s /tmp/plasma -m 1000000000`` from the command line and will start the plasma_store executable with the given arguments. + + .. deprecated:: 10.0.0 + Plasma is deprecated since Arrow 10.0.0. It will be removed in 12.0.0 or so. """ + warnings.warn( + "Plasma is deprecated since Arrow 10.0.0. It will be removed in 12.0.0 or so.", + DeprecationWarning) + import pyarrow plasma_store_executable = _os.path.join(pyarrow.__path__[0], "plasma-store-server") diff --git a/python/pyarrow/_plasma.pyx b/python/pyarrow/_plasma.pyx index 35d39073634e0..61a4ac14651e6 100644 --- a/python/pyarrow/_plasma.pyx +++ b/python/pyarrow/_plasma.pyx @@ -156,7 +156,10 @@ def make_object_id(object_id): cdef class ObjectID(_Weakrefable): """ - An ObjectID represents a string of bytes used to identify Plasma objects. + DEPRECATED: An ObjectID represents a string of bytes used to identify Plasma objects. + + .. deprecated:: 10.0.0 + Plasma is deprecated since Arrow 10.0.0. It will be removed in 12.0.0 or so. """ cdef: @@ -169,6 +172,10 @@ cdef class ObjectID(_Weakrefable): " is " + str(object_id)) self.data = CUniqueID.from_binary(object_id) + warnings.warn( + "Plasma is deprecated since Arrow 10.0.0. It will be removed in 12.0.0 or so.", + DeprecationWarning, stacklevel=2) + def __eq__(self, other): try: return self.data == (other).data @@ -219,13 +226,16 @@ cdef class ObjectNotAvailable(_Weakrefable): cdef class PlasmaBuffer(Buffer): """ - This is the type returned by calls to get with a PlasmaClient. + DEPRECATED: This is the type returned by calls to get with a PlasmaClient. We define our own class instead of directly returning a buffer object so that we can add a custom destructor which notifies Plasma that the object is no longer being used, so the memory in the Plasma store backing the object can potentially be freed. + .. deprecated:: 10.0.0 + Plasma is deprecated since Arrow 10.0.0. It will be removed in 12.0.0 or so. + Attributes ---------- object_id : ObjectID @@ -295,11 +305,14 @@ def get_socket_from_fd(fileno, family, type): cdef class PlasmaClient(_Weakrefable): """ - The PlasmaClient is used to interface with a plasma store and manager. + DEPRECATED: The PlasmaClient is used to interface with a plasma store and manager. The PlasmaClient can ask the PlasmaStore to allocate a new buffer, seal a buffer, and get a buffer. Buffers are referred to by object IDs, which are strings. + + .. deprecated:: 10.0.0 + Plasma is deprecated since Arrow 10.0.0. It will be removed in 12.0.0 or so. """ cdef: @@ -312,6 +325,10 @@ cdef class PlasmaClient(_Weakrefable): self.notification_fd = -1 self.store_socket_name = b"" + warnings.warn( + "Plasma is deprecated since Arrow 10.0.0. It will be removed in 12.0.0 or so.", + DeprecationWarning, stacklevel=3) + cdef _get_object_buffers(self, object_ids, int64_t timeout_ms, c_vector[CObjectBuffer]* result): cdef: @@ -854,9 +871,12 @@ cdef class PlasmaClient(_Weakrefable): def connect(store_socket_name, int num_retries=-1): """ - Return a new PlasmaClient that is connected a plasma store and + DEPRECATED: Return a new PlasmaClient that is connected a plasma store and optionally a manager. + .. deprecated:: 10.0.0 + Plasma is deprecated since Arrow 10.0.0. It will be removed in 12.0.0 or so. + Parameters ---------- store_socket_name : str diff --git a/python/pyarrow/plasma.py b/python/pyarrow/plasma.py index 712ed9e5b2261..5c2c6543418d4 100644 --- a/python/pyarrow/plasma.py +++ b/python/pyarrow/plasma.py @@ -24,6 +24,7 @@ import sys import tempfile import time +import warnings from pyarrow._plasma import (ObjectID, ObjectNotAvailable, # noqa PlasmaBuffer, PlasmaClient, connect, @@ -84,7 +85,11 @@ def start_plasma_store(plasma_store_memory, plasma_directory=None, use_hugepages=False, external_store=None): """ - Start a plasma store process. + DEPRECATED: Start a plasma store process. + + .. deprecated:: 10.0.0 + Plasma is deprecated since Arrow 10.0.0. It will be removed + in 12.0.0 or so. Parameters ---------- @@ -109,6 +114,11 @@ def start_plasma_store(plasma_store_memory, A tuple of the name of the plasma store socket and the process ID of the plasma store process. """ + warnings.warn( + "Plasma is deprecated since Arrow 10.0.0. It will be removed in " + "12.0.0 or so.", + DeprecationWarning) + if use_valgrind and use_profiler: raise Exception("Cannot use valgrind and profiler at the same time.") diff --git a/python/pyarrow/src/arrow/python/arrow_to_pandas.cc b/python/pyarrow/src/arrow/python/arrow_to_pandas.cc index af778f5a8fabb..6e12bb1962db1 100644 --- a/python/pyarrow/src/arrow/python/arrow_to_pandas.cc +++ b/python/pyarrow/src/arrow/python/arrow_to_pandas.cc @@ -191,6 +191,10 @@ static inline bool ListTypeSupported(const DataType& type) { const auto& list_type = checked_cast(type); return ListTypeSupported(*list_type.value_type()); } + case Type::EXTENSION: { + const auto& ext = checked_cast(*type.GetSharedPtr()); + return ListTypeSupported(*(ext.storage_type())); + } default: break; } @@ -734,11 +738,20 @@ Status ConvertListsLike(PandasOptions options, const ChunkedArray& data, ArrayVector value_arrays; for (int c = 0; c < data.num_chunks(); c++) { const auto& arr = checked_cast(*data.chunk(c)); - value_arrays.emplace_back(arr.values()); + if (arr.value_type()->id() == Type::EXTENSION) { + const auto& arr_ext = checked_cast(*arr.values()); + value_arrays.emplace_back(arr_ext.storage()); + } else { + value_arrays.emplace_back(arr.values()); + } } + using ListArrayType = typename ListArrayT::TypeClass; const auto& list_type = checked_cast(*data.type()); auto value_type = list_type.value_type(); + if (value_type->id() == Type::EXTENSION) { + value_type = checked_cast(*value_type).storage_type(); + } auto flat_column = std::make_shared(value_arrays, value_type); @@ -747,14 +760,12 @@ Status ConvertListsLike(PandasOptions options, const ChunkedArray& data, OwnedRefNoGIL owned_numpy_array; RETURN_NOT_OK(ConvertChunkedArrayToPandas(options, flat_column, nullptr, owned_numpy_array.ref())); - PyObject* numpy_array = owned_numpy_array.obj(); DCHECK(PyArray_Check(numpy_array)); int64_t chunk_offset = 0; for (int c = 0; c < data.num_chunks(); c++) { const auto& arr = checked_cast(*data.chunk(c)); - const bool has_nulls = data.null_count() > 0; for (int64_t i = 0; i < arr.length(); ++i) { if (has_nulls && arr.IsNull(i)) { diff --git a/python/pyarrow/tests/test_extension_type.py b/python/pyarrow/tests/test_extension_type.py index caa3f5d4f012b..f5723491cb4e7 100644 --- a/python/pyarrow/tests/test_extension_type.py +++ b/python/pyarrow/tests/test_extension_type.py @@ -78,6 +78,15 @@ def __reduce__(self): return UuidType2, () +class LabelType(pa.PyExtensionType): + + def __init__(self): + pa.PyExtensionType.__init__(self, pa.string()) + + def __reduce__(self): + return LabelType, () + + class ParamExtType(pa.PyExtensionType): def __init__(self, width): @@ -1020,6 +1029,26 @@ def test_empty_take(): assert result.equals(empty_arr) +@pytest.mark.parametrize("data,ty", ( + ([1, 2, 3], IntegerType), + (["cat", "dog", "horse"], LabelType) +)) +@pytest.mark.parametrize("into", ("to_numpy", "to_pandas")) +def test_extension_array_to_numpy_pandas(data, ty, into): + storage = pa.array(data) + ext_arr = pa.ExtensionArray.from_storage(ty(), storage) + offsets = pa.array([0, 1, 2, 3]) + list_arr = pa.ListArray.from_arrays(offsets, ext_arr) + result = getattr(list_arr, into)(zero_copy_only=False) + + list_arr_storage_type = list_arr.cast(pa.list_(ext_arr.type.storage_type)) + expected = getattr(list_arr_storage_type, into)(zero_copy_only=False) + if into == "to_pandas": + assert result.equals(expected) + else: + assert np.array_equal(result, expected) + + def test_array_constructor(): ext_type = IntegerType() storage = pa.array([1, 2, 3], type=pa.int64()) diff --git a/python/pyarrow/tests/test_plasma.py b/python/pyarrow/tests/test_plasma.py index ed08a6872582b..cc5fd0357ddee 100644 --- a/python/pyarrow/tests/test_plasma.py +++ b/python/pyarrow/tests/test_plasma.py @@ -30,6 +30,10 @@ import pyarrow as pa +# ignore all Plasma deprecation warnings in this file, we test that the +# warnings are actually raised in test_plasma_deprecated.py +pytestmark = pytest.mark.filterwarnings("ignore:Plasma:DeprecationWarning") + DEFAULT_PLASMA_STORE_MEMORY = 10 ** 8 USE_VALGRIND = os.getenv("PLASMA_VALGRIND") == "1" EXTERNAL_STORE = "hashtable://test" @@ -1071,3 +1075,27 @@ def test_store_capacity(): with plasma.start_plasma_store(plasma_store_memory=10000) as (name, p): plasma_client = plasma.connect(name) assert plasma_client.store_capacity() == 10000 + + +@pytest.mark.plasma +def test_plasma_deprecated(): + import pyarrow.plasma as plasma + + plasma_store_ctx = plasma.start_plasma_store( + plasma_store_memory=10 ** 8, + use_valgrind=os.getenv("PLASMA_VALGRIND") == "1") + + with pytest.warns(DeprecationWarning): + with plasma_store_ctx: + pass + + plasma_store_ctx = plasma.start_plasma_store( + plasma_store_memory=10 ** 8, + use_valgrind=os.getenv("PLASMA_VALGRIND") == "1") + + with plasma_store_ctx as (plasma_store_name, _): + with pytest.warns(DeprecationWarning): + plasma.connect(plasma_store_name) + + with pytest.warns(DeprecationWarning): + plasma.ObjectID(20 * b"a") diff --git a/python/pyarrow/tests/test_substrait.py b/python/pyarrow/tests/test_substrait.py index d7d5d81a19e24..a1451ba550551 100644 --- a/python/pyarrow/tests/test_substrait.py +++ b/python/pyarrow/tests/test_substrait.py @@ -16,7 +16,6 @@ # under the License. import os -import sys import pytest import pyarrow as pa @@ -40,9 +39,6 @@ def _write_dummy_data_to_disk(tmpdir, file_name, table): return path -@pytest.mark.skipif(sys.platform == 'win32', - reason="ARROW-16392: file based URI is" + - " not fully supported for Windows") def test_run_serialized_query(tmpdir): substrait_query = """ { @@ -115,9 +111,6 @@ def test_invalid_plan(): substrait.run_query(buf) -@pytest.mark.skipif(sys.platform == 'win32', - reason="ARROW-16392: file based URI is" + - " not fully supported for Windows") def test_binary_conversion_with_json_options(tmpdir): substrait_query = """ { diff --git a/r/R/dplyr-across.R b/r/R/dplyr-across.R index d23525ddfb56b..5b816a0719976 100644 --- a/r/R/dplyr-across.R +++ b/r/R/dplyr-across.R @@ -123,8 +123,19 @@ across_setup <- function(cols, fns, names, .caller_env, mask, inline = FALSE) { return(value) } + is_single_func <- function(fns) { + # function calls with package, like base::round + (is.call(fns) && fns[[1]] == as.name("::")) || + # purrr-style formulae + is_formula(fns) || + # single anonymous function + is_call(fns, "function") || + # any other length 1 function calls + (length(fns) == 1 && (is.function(fns) || is_formula(fns) || is.name(fns))) + } + # apply `.names` smart default - if (is.function(fns) || is_formula(fns) || is.name(fns) || is_call(fns, "function")) { + if (is_single_func(fns)) { names <- names %||% "{.col}" fns <- list("1" = fns) } else { @@ -193,13 +204,13 @@ as_across_fn_call <- function(fn, var, quo_env) { expr_substitute <- function(expr, old, new) { switch(typeof(expr), - language = { - expr[] <- lapply(expr, expr_substitute, old, new) - return(expr) - }, - symbol = if (identical(expr, old)) { - return(new) - } + language = { + expr[] <- lapply(expr, expr_substitute, old, new) + return(expr) + }, + symbol = if (identical(expr, old)) { + return(new) + } ) expr diff --git a/r/R/dplyr-collect.R b/r/R/dplyr-collect.R index a3ea2fc0c58f3..5640565b15385 100644 --- a/r/R/dplyr-collect.R +++ b/r/R/dplyr-collect.R @@ -50,9 +50,13 @@ pull.arrow_dplyr_query <- function(.data, var = -1) { .data <- as_adq(.data) var <- vars_pull(names(.data), !!enquo(var)) .data$selected_columns <- set_names(.data$selected_columns[var], var) - dplyr::collect(.data)[[1]] + dplyr::compute(.data)[[1]] +} +pull.Dataset <- pull.RecordBatchReader <- pull.arrow_dplyr_query + +pull.ArrowTabular <- function(x, var = -1) { + x[[vars_pull(names(x), !!enquo(var))]] } -pull.Dataset <- pull.ArrowTabular <- pull.RecordBatchReader <- pull.arrow_dplyr_query restore_dplyr_features <- function(df, query) { # An arrow_dplyr_query holds some attributes that Arrow doesn't know about diff --git a/r/tests/testthat/test-dataset-write.R b/r/tests/testthat/test-dataset-write.R index 7a5f861ca579d..468d11676bb2a 100644 --- a/r/tests/testthat/test-dataset-write.R +++ b/r/tests/testthat/test-dataset-write.R @@ -739,7 +739,8 @@ test_that("Dataset min_rows_per_group", { row_group_sizes <- ds %>% map_batches(~ record_batch(nrows = .$num_rows)) %>% - pull(nrows) + pull(nrows) %>% + as.vector() index <- 1 # We expect there to be 3 row groups since 11/5 = 2.2 and 11/4 = 2.75 @@ -778,6 +779,7 @@ test_that("Dataset write max rows per group", { row_group_sizes <- ds %>% map_batches(~ record_batch(nrows = .$num_rows)) %>% pull(nrows) %>% + as.vector() %>% sort() expect_equal(row_group_sizes, c(12, 18)) diff --git a/r/tests/testthat/test-dataset.R b/r/tests/testthat/test-dataset.R index ea3e1365d6bc0..5e28a3ddd5bab 100644 --- a/r/tests/testthat/test-dataset.R +++ b/r/tests/testthat/test-dataset.R @@ -67,7 +67,7 @@ test_that("IPC/Feather format data", { # Collecting virtual partition column works expect_equal( - ds %>% arrange(part) %>% pull(part), + ds %>% arrange(part) %>% pull(part) %>% as.vector(), c(rep(3, 10), rep(4, 10)) ) }) @@ -306,7 +306,7 @@ test_that("Simple interface for datasets", { # Collecting virtual partition column works expect_equal( - ds %>% arrange(part) %>% pull(part), + ds %>% arrange(part) %>% pull(part) %>% as.vector(), c(rep(1, 10), rep(2, 10)) ) }) @@ -625,8 +625,16 @@ test_that("scalar aggregates with many batches (ARROW-16904)", { ds <- open_dataset(tf) replicate(100, ds %>% summarize(min(x)) %>% pull()) - expect_true(all(replicate(100, ds %>% summarize(min(x)) %>% pull()) == 1)) - expect_true(all(replicate(100, ds %>% summarize(max(x)) %>% pull()) == 100)) + expect_true( + all( + replicate(100, ds %>% summarize(min(x)) %>% pull() %>% as.vector()) == 1 + ) + ) + expect_true( + all( + replicate(100, ds %>% summarize(max(x)) %>% pull() %>% as.vector()) == 100 + ) + ) }) test_that("map_batches", { @@ -650,6 +658,7 @@ test_that("map_batches", { select(int, lgl) %>% map_batches(~ record_batch(nrows = .$num_rows)) %>% pull(nrows) %>% + as.vector() %>% sort(), c(5, 10) ) @@ -1170,7 +1179,8 @@ test_that("FileSystemFactoryOptions with DirectoryPartitioning", { expect_equal( ds %>% arrange(cyl) %>% - pull(cyl), + pull(cyl) %>% + as.vector(), sort(mtcars$cyl) ) @@ -1188,7 +1198,8 @@ test_that("FileSystemFactoryOptions with DirectoryPartitioning", { expect_equal( ds %>% arrange(cyl) %>% - pull(cyl), + pull(cyl) %>% + as.vector(), sort(mtcars$cyl) ) @@ -1204,7 +1215,8 @@ test_that("FileSystemFactoryOptions with DirectoryPartitioning", { expect_equal( ds %>% arrange(cyl) %>% - pull(cyl), + pull(cyl) %>% + as.vector(), sort(mtcars$cyl) ) @@ -1222,7 +1234,8 @@ test_that("FileSystemFactoryOptions with DirectoryPartitioning", { expect_equal( ds %>% arrange(cyl) %>% - pull(cyl), + pull(cyl) %>% + as.vector(), sort(mtcars$cyl) ) @@ -1256,7 +1269,8 @@ test_that("FileSystemFactoryOptions with HivePartitioning", { expect_equal( ds %>% arrange(cyl) %>% - pull(cyl), + pull(cyl) %>% + as.vector(), sort(mtcars$cyl) ) @@ -1272,7 +1286,8 @@ test_that("FileSystemFactoryOptions with HivePartitioning", { expect_equal( ds %>% arrange(cyl) %>% - pull(cyl), + pull(cyl) %>% + as.vector(), sort(mtcars$cyl) ) @@ -1286,7 +1301,8 @@ test_that("FileSystemFactoryOptions with HivePartitioning", { expect_equal( ds %>% arrange(cyl) %>% - pull(cyl), + pull(cyl) %>% + as.vector(), sort(mtcars$cyl) ) @@ -1302,7 +1318,8 @@ test_that("FileSystemFactoryOptions with HivePartitioning", { expect_equal( ds %>% arrange(cyl) %>% - pull(cyl), + pull(cyl) %>% + as.vector(), sort(mtcars$cyl) ) }) diff --git a/r/tests/testthat/test-dplyr-across.R b/r/tests/testthat/test-dplyr-across.R index 5ded2038c4fc5..edf74dcbdb7a2 100644 --- a/r/tests/testthat/test-dplyr-across.R +++ b/r/tests/testthat/test-dplyr-across.R @@ -212,6 +212,27 @@ test_that("expand_across correctly expands quosures", { regexp = "`.names` specification must produce (number of columns * number of functions) names.", fixed = TRUE ) + + # Using package name prefix (ARROW-17724) + expect_across_equal( + quos(across(c(dbl, dbl2), base::round)), + quos( + dbl = base::round(dbl), + dbl2 = base::round(dbl2) + ), + example_data + ) + + expect_across_equal( + quos(across(c(dbl, dbl2), c(base::round, base::sqrt))), + quos( + dbl_1 = base::round(dbl), + dbl_2 = base::sqrt(dbl), + dbl2_1 = base::round(dbl2), + dbl2_2 = base::sqrt(dbl2) + ), + example_data + ) }) test_that("purrr-style lambda functions are supported", { diff --git a/r/tests/testthat/test-dplyr-arrange.R b/r/tests/testthat/test-dplyr-arrange.R index d8afcc5d4a80d..3444e3ace5f22 100644 --- a/r/tests/testthat/test-dplyr-arrange.R +++ b/r/tests/testthat/test-dplyr-arrange.R @@ -118,7 +118,8 @@ test_that("arrange() on integer, double, and character columns", { .input %>% group_by(grp) %>% arrange(.by_group = TRUE) %>% - pull(grp), + pull(grp) %>% + as.vector(), tbl ) compare_dplyr_binding( diff --git a/r/tests/testthat/test-dplyr-funcs-datetime.R b/r/tests/testthat/test-dplyr-funcs-datetime.R index 25a844e033384..2608f9d65454a 100644 --- a/r/tests/testthat/test-dplyr-funcs-datetime.R +++ b/r/tests/testthat/test-dplyr-funcs-datetime.R @@ -168,7 +168,8 @@ test_that("strptime", { mutate( x = strptime(x, format = "%m-%d-%Y") ) %>% - pull(), + pull() %>% + as.vector(), # R's strptime returns POSIXlt (list type) as.POSIXct(tstamp), ignore_attr = "tzone" diff --git a/r/tests/testthat/test-dplyr-query.R b/r/tests/testthat/test-dplyr-query.R index c40815df69d85..db9a3bb30d00b 100644 --- a/r/tests/testthat/test-dplyr-query.R +++ b/r/tests/testthat/test-dplyr-query.R @@ -70,22 +70,23 @@ See $.data for the source Arrow object', test_that("pull", { compare_dplyr_binding( - .input %>% pull(), + .input %>% pull() %>% as.vector(), tbl ) compare_dplyr_binding( - .input %>% pull(1), + .input %>% pull(1) %>% as.vector(), tbl ) compare_dplyr_binding( - .input %>% pull(chr), + .input %>% pull(chr) %>% as.vector(), tbl ) compare_dplyr_binding( .input %>% filter(int > 4) %>% rename(strng = chr) %>% - pull(strng), + pull(strng) %>% + as.vector(), tbl ) })