From 09b6b788c62d1a7671175661778dd215db7a2779 Mon Sep 17 00:00:00 2001 From: Gang Wu Date: Thu, 22 Sep 2022 16:37:46 +0800 Subject: [PATCH] ARROW-17814: [C++] Remove make_unique reimplementation Use std::make_unique instead. --- c_glib/arrow-flight-glib/server.cpp | 10 ++--- c_glib/arrow-flight-sql-glib/server.cpp | 6 +-- cpp/src/arrow/buffer_test.cc | 3 +- cpp/src/arrow/builder.cc | 18 ++++---- cpp/src/arrow/compute/exec.cc | 7 ++-- cpp/src/arrow/compute/exec/asof_join_node.cc | 6 +-- .../arrow/compute/exec/asof_join_node_test.cc | 4 +- cpp/src/arrow/compute/exec/expression_test.cc | 3 +- .../arrow/compute/exec/hash_join_benchmark.cc | 5 +-- cpp/src/arrow/compute/exec/hash_join_node.cc | 8 ++-- .../arrow/compute/exec/hash_join_node_test.cc | 20 ++++----- cpp/src/arrow/compute/exec/plan_test.cc | 5 +-- cpp/src/arrow/compute/exec/tpch_benchmark.cc | 3 +- cpp/src/arrow/compute/exec/tpch_node.cc | 11 +++-- cpp/src/arrow/compute/exec/tpch_node_test.cc | 2 +- cpp/src/arrow/compute/exec_test.cc | 3 +- .../arrow/compute/kernels/aggregate_basic.cc | 11 ++--- .../arrow/compute/kernels/codegen_internal.h | 5 +-- .../arrow/compute/kernels/hash_aggregate.cc | 5 +-- cpp/src/arrow/compute/kernels/row_encoder.cc | 3 +- .../compute/kernels/scalar_arithmetic.cc | 6 +-- .../compute/kernels/scalar_string_ascii.cc | 14 +++---- .../compute/kernels/vector_cumulative_ops.cc | 4 +- cpp/src/arrow/compute/kernels/vector_hash.cc | 6 +-- .../compute/kernels/vector_replace_test.cc | 5 ++- cpp/src/arrow/compute/registry_test.cc | 3 +- cpp/src/arrow/compute/row/grouper.cc | 18 ++++---- cpp/src/arrow/csv/chunker.cc | 3 +- cpp/src/arrow/csv/writer.cc | 3 +- cpp/src/arrow/dataset/dataset.cc | 1 - cpp/src/arrow/dataset/dataset_writer.cc | 10 ++--- cpp/src/arrow/dataset/file_base.cc | 2 +- cpp/src/arrow/dataset/file_csv_test.cc | 4 +- cpp/src/arrow/dataset/partition.cc | 3 +- cpp/src/arrow/dataset/test_util.h | 5 +-- .../engine/substrait/expression_internal.cc | 42 +++++++++---------- .../arrow/engine/substrait/extension_set.cc | 4 +- .../arrow/engine/substrait/plan_internal.cc | 20 ++++----- .../engine/substrait/relation_internal.cc | 5 ++- .../engine/substrait/test_plan_builder.cc | 4 +- .../arrow/engine/substrait/type_internal.cc | 14 +++---- cpp/src/arrow/filesystem/localfs_benchmark.cc | 3 +- cpp/src/arrow/flight/client.cc | 11 +++-- cpp/src/arrow/flight/cookie_internal.cc | 3 +- cpp/src/arrow/flight/flight_test.cc | 13 +++--- cpp/src/arrow/flight/test_util.h | 3 +- cpp/src/arrow/flight/transport.cc | 4 +- .../flight/transport/grpc/grpc_client.cc | 3 +- .../arrow/flight/transport/ucx/ucx_client.cc | 10 ++--- .../flight/transport/ucx/ucx_internal.cc | 8 ++-- .../arrow/flight/transport/ucx/ucx_server.cc | 4 +- .../flight/transport/ucx/util_internal.cc | 2 +- cpp/src/arrow/flight/types.cc | 3 +- cpp/src/arrow/gpu/cuda_context.cc | 3 +- cpp/src/arrow/ipc/feather.cc | 3 +- cpp/src/arrow/ipc/reader.cc | 2 +- cpp/src/arrow/ipc/writer.cc | 18 ++++---- cpp/src/arrow/json/chunker.cc | 2 - cpp/src/arrow/json/parser.cc | 4 +- cpp/src/arrow/type.cc | 8 ++-- cpp/src/arrow/util/async_util.cc | 12 +++--- cpp/src/arrow/util/async_util.h | 5 ++- cpp/src/arrow/util/async_util_test.cc | 8 ++-- cpp/src/arrow/util/converter.h | 6 +-- cpp/src/arrow/util/future_test.cc | 3 +- cpp/src/arrow/util/make_unique.h | 42 ------------------- cpp/src/arrow/util/string_builder.cc | 7 +--- cpp/src/arrow/util/tracing.cc | 5 ++- cpp/src/arrow/util/tracing_internal.cc | 12 +++--- cpp/src/arrow/util/tracing_internal.h | 1 - cpp/src/gandiva/engine.cc | 7 ++-- cpp/src/parquet/arrow/path_internal.cc | 5 +-- cpp/src/parquet/arrow/reader.cc | 6 +-- cpp/src/parquet/arrow/writer.cc | 6 +-- cpp/src/parquet/column_reader_test.cc | 5 +-- cpp/src/parquet/reader_test.cc | 3 +- cpp/src/skyhook/protocol/rados_protocol.h | 11 ++--- 77 files changed, 250 insertions(+), 315 deletions(-) delete mode 100644 cpp/src/arrow/util/make_unique.h diff --git a/c_glib/arrow-flight-glib/server.cpp b/c_glib/arrow-flight-glib/server.cpp index 40bad8b496faf..4af1bf60d4787 100644 --- a/c_glib/arrow-flight-glib/server.cpp +++ b/c_glib/arrow-flight-glib/server.cpp @@ -17,7 +17,7 @@ * under the License. */ -#include +#include #include @@ -239,7 +239,7 @@ gaflight_record_batch_stream_new(GArrowRecordBatchReader *reader, } else { arrow_options = &arrow_options_default; } - auto stream = arrow::internal::make_unique< + auto stream = std::make_unique< arrow::flight::RecordBatchStream>(arrow_reader, *arrow_options); return static_cast( g_object_new(GAFLIGHT_TYPE_RECORD_BATCH_STREAM, @@ -484,7 +484,7 @@ namespace gaflight { g_object_unref(gaflight); } g_list_free(gaflights); - *listing = arrow::internal::make_unique< + *listing = std::make_unique< arrow::flight::SimpleFlightListing>(flights); return arrow::Status::OK(); } @@ -507,7 +507,7 @@ namespace gaflight { arrow::StatusCode::UnknownError, "[flight-server][get-flight-info]"); } - *info = arrow::internal::make_unique( + *info = std::make_unique( *gaflight_info_get_raw(gainfo)); g_object_unref(gainfo); return arrow::Status::OK(); @@ -531,7 +531,7 @@ namespace gaflight { arrow::StatusCode::UnknownError, "[flight-server][do-get]"); } - *stream = arrow::internal::make_unique(gastream); + *stream = std::make_unique(gastream); return arrow::Status::OK(); } diff --git a/c_glib/arrow-flight-sql-glib/server.cpp b/c_glib/arrow-flight-sql-glib/server.cpp index 32fdc85e9bfa7..51cdb22ab5d2b 100644 --- a/c_glib/arrow-flight-sql-glib/server.cpp +++ b/c_glib/arrow-flight-sql-glib/server.cpp @@ -17,7 +17,7 @@ * under the License. */ -#include +#include #include #include @@ -225,7 +225,7 @@ namespace gaflightsql { arrow::StatusCode::UnknownError, context); } - return arrow::internal::make_unique( + return std::make_unique( *gaflight_info_get_raw(gainfo)); } @@ -247,7 +247,7 @@ namespace gaflightsql { arrow::StatusCode::UnknownError, "[flight-sql-server][do-get-statement]"); } - return arrow::internal::make_unique(gastream); + return std::make_unique(gastream); } private: diff --git a/cpp/src/arrow/buffer_test.cc b/cpp/src/arrow/buffer_test.cc index fd159dd979793..5712def4926cc 100644 --- a/cpp/src/arrow/buffer_test.cc +++ b/cpp/src/arrow/buffer_test.cc @@ -34,7 +34,6 @@ #include "arrow/status.h" #include "arrow/testing/gtest_util.h" #include "arrow/util/checked_cast.h" -#include "arrow/util/make_unique.h" namespace arrow { @@ -162,7 +161,7 @@ Result> MyMemoryManager::CopyNonOwnedFrom( ARROW_ASSIGN_OR_RAISE(auto dest, MemoryManager::CopyNonOwned(buf, default_cpu_memory_manager())); // 2. Wrap CPU buffer result - return internal::make_unique(shared_from_this(), std::move(dest)); + return std::make_unique(shared_from_this(), std::move(dest)); } return nullptr; } diff --git a/cpp/src/arrow/builder.cc b/cpp/src/arrow/builder.cc index 779722e0d1c78..43a1c6d8a30f5 100644 --- a/cpp/src/arrow/builder.cc +++ b/cpp/src/arrow/builder.cc @@ -17,6 +17,7 @@ #include "arrow/builder.h" +#include #include #include #include @@ -25,7 +26,6 @@ #include "arrow/type.h" #include "arrow/util/checked_cast.h" #include "arrow/util/hashing.h" -#include "arrow/util/make_unique.h" #include "arrow/visit_type_inline.h" namespace arrow { @@ -54,28 +54,28 @@ class ARROW_EXPORT TypeErasedIntBuilder : public ArrayBuilder { DCHECK(is_integer(type_id_)); switch (type_id_) { case Type::UINT8: - builder_ = internal::make_unique(pool); + builder_ = std::make_unique(pool); break; case Type::INT8: - builder_ = internal::make_unique(pool); + builder_ = std::make_unique(pool); break; case Type::UINT16: - builder_ = internal::make_unique(pool); + builder_ = std::make_unique(pool); break; case Type::INT16: - builder_ = internal::make_unique(pool); + builder_ = std::make_unique(pool); break; case Type::UINT32: - builder_ = internal::make_unique(pool); + builder_ = std::make_unique(pool); break; case Type::INT32: - builder_ = internal::make_unique(pool); + builder_ = std::make_unique(pool); break; case Type::UINT64: - builder_ = internal::make_unique(pool); + builder_ = std::make_unique(pool); break; case Type::INT64: - builder_ = internal::make_unique(pool); + builder_ = std::make_unique(pool); break; default: DCHECK(false); diff --git a/cpp/src/arrow/compute/exec.cc b/cpp/src/arrow/compute/exec.cc index cf91bada6c6f7..b0bec25582825 100644 --- a/cpp/src/arrow/compute/exec.cc +++ b/cpp/src/arrow/compute/exec.cc @@ -47,7 +47,6 @@ #include "arrow/util/checked_cast.h" #include "arrow/util/cpu_info.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/vector.h" namespace arrow { @@ -1187,15 +1186,15 @@ void PropagateNullsSpans(const ExecSpan& batch, ArraySpan* out) { } std::unique_ptr KernelExecutor::MakeScalar() { - return ::arrow::internal::make_unique(); + return std::make_unique(); } std::unique_ptr KernelExecutor::MakeVector() { - return ::arrow::internal::make_unique(); + return std::make_unique(); } std::unique_ptr KernelExecutor::MakeScalarAggregate() { - return ::arrow::internal::make_unique(); + return std::make_unique(); } int64_t InferBatchLength(const std::vector& values, bool* all_same) { diff --git a/cpp/src/arrow/compute/exec/asof_join_node.cc b/cpp/src/arrow/compute/exec/asof_join_node.cc index 09ef7d722d9ac..51643488a8765 100644 --- a/cpp/src/arrow/compute/exec/asof_join_node.cc +++ b/cpp/src/arrow/compute/exec/asof_join_node.cc @@ -16,6 +16,7 @@ // under the License. #include +#include #include #include #include @@ -37,7 +38,6 @@ #include "arrow/type_traits.h" #include "arrow/util/checked_cast.h" #include "arrow/util/future.h" -#include "arrow/util/make_unique.h" namespace arrow { namespace compute { @@ -850,7 +850,7 @@ class AsofJoinNode : public ExecNode { auto inputs = this->inputs(); for (size_t i = 0; i < inputs.size(); i++) { RETURN_NOT_OK(key_hashers_[i]->Init(plan()->exec_context(), output_schema())); - state_.push_back(::arrow::internal::make_unique( + state_.push_back(std::make_unique( must_hash_, may_rehash_, key_hashers_[i].get(), inputs[i]->output_schema(), indices_of_on_key_[i], indices_of_by_key_[i])); } @@ -1060,7 +1060,7 @@ class AsofJoinNode : public ExecNode { std::vector> key_hashers; for (size_t i = 0; i < n_input; i++) { key_hashers.push_back( - ::arrow::internal::make_unique(indices_of_by_key[i])); + std::make_unique(indices_of_by_key[i])); } bool must_hash = n_by > 1 || diff --git a/cpp/src/arrow/compute/exec/asof_join_node_test.cc b/cpp/src/arrow/compute/exec/asof_join_node_test.cc index 919bfdbde497c..466c111bb91e8 100644 --- a/cpp/src/arrow/compute/exec/asof_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/asof_join_node_test.cc @@ -18,6 +18,7 @@ #include #include +#include #include #include #include @@ -34,7 +35,6 @@ #include "arrow/testing/matchers.h" #include "arrow/testing/random.h" #include "arrow/util/checked_cast.h" -#include "arrow/util/make_unique.h" #include "arrow/util/thread_pool.h" #define TRACED_TEST(t_class, t_name, t_body) \ @@ -210,7 +210,7 @@ void CheckRunOutput(const BatchesWithSchema& l_batches, const BatchesWithSchema& exp_batches, const AsofJoinNodeOptions join_options) { auto exec_ctx = - arrow::internal::make_unique(default_memory_pool(), nullptr); + std::make_unique(default_memory_pool(), nullptr); ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); Declaration join{"asofjoin", join_options}; diff --git a/cpp/src/arrow/compute/exec/expression_test.cc b/cpp/src/arrow/compute/exec/expression_test.cc index 4cb4c272485d2..25b38cc80ff70 100644 --- a/cpp/src/arrow/compute/exec/expression_test.cc +++ b/cpp/src/arrow/compute/exec/expression_test.cc @@ -30,7 +30,6 @@ #include "arrow/compute/function_internal.h" #include "arrow/compute/registry.h" #include "arrow/testing/gtest_util.h" -#include "arrow/util/make_unique.h" using testing::HasSubstr; using testing::UnorderedElementsAreArray; @@ -242,7 +241,7 @@ class WidgetifyOptionsType : public FunctionOptionsType { } std::unique_ptr Copy(const FunctionOptions& options) const override { const auto& opts = static_cast(options); - return arrow::internal::make_unique(opts.really); + return std::make_unique(opts.really); } }; WidgetifyOptions::WidgetifyOptions(bool really) diff --git a/cpp/src/arrow/compute/exec/hash_join_benchmark.cc b/cpp/src/arrow/compute/exec/hash_join_benchmark.cc index 94201a849fa43..657ee804f6a63 100644 --- a/cpp/src/arrow/compute/exec/hash_join_benchmark.cc +++ b/cpp/src/arrow/compute/exec/hash_join_benchmark.cc @@ -25,7 +25,6 @@ #include "arrow/compute/exec/util.h" #include "arrow/compute/kernels/row_encoder.h" #include "arrow/testing/random.h" -#include "arrow/util/make_unique.h" #include "arrow/util/thread_pool.h" #include @@ -126,10 +125,10 @@ class JoinBenchmark { stats_.num_probe_rows = settings.num_probe_batches * settings.batch_size; - ctx_ = arrow::internal::make_unique(default_memory_pool(), + ctx_ = std::make_unique(default_memory_pool(), arrow::internal::GetCpuThreadPool()); - schema_mgr_ = arrow::internal::make_unique(); + schema_mgr_ = std::make_unique(); Expression filter = literal(true); DCHECK_OK(schema_mgr_->Init(settings.join_type, *l_batches_with_schema.schema, left_keys, *r_batches_with_schema.schema, right_keys, diff --git a/cpp/src/arrow/compute/exec/hash_join_node.cc b/cpp/src/arrow/compute/exec/hash_join_node.cc index 237f13d0a258a..9759101585d50 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node.cc @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +#include #include #include #include @@ -29,7 +30,6 @@ #include "arrow/compute/exec/util.h" #include "arrow/util/checked_cast.h" #include "arrow/util/future.h" -#include "arrow/util/make_unique.h" #include "arrow/util/thread_pool.h" #include "arrow/util/tracing_internal.h" @@ -580,7 +580,7 @@ struct BloomFilterPushdownContext { selected.data()); } auto selected_buffer = - arrow::internal::make_unique(selected.data(), bit_vector_bytes); + std::make_unique(selected.data(), bit_vector_bytes); ArrayData selected_arraydata(boolean(), batch.length, {nullptr, std::move(selected_buffer)}); Datum selected_datum(selected_arraydata); @@ -716,7 +716,7 @@ class HashJoinNode : public ExecNode { RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 2, "HashJoinNode")); std::unique_ptr schema_mgr = - ::arrow::internal::make_unique(); + std::make_unique(); const auto& join_options = checked_cast(options); RETURN_NOT_OK(ValidateHashJoinNodeOptions(join_options)); @@ -1093,7 +1093,7 @@ void BloomFilterPushdownContext::Init( eval_.all_received_callback_ = std::move(on_bloom_filters_received); if (!disable_bloom_filter_) { ARROW_CHECK(push_.pushdown_target_); - push_.bloom_filter_ = arrow::internal::make_unique(); + push_.bloom_filter_ = std::make_unique(); push_.pushdown_target_->pushdown_context_.ExpectBloomFilter(); build_.builder_ = BloomFilterBuilder::Make( diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc index de3592ab086aa..ec6357fc80683 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -17,6 +17,7 @@ #include +#include #include #include #include @@ -32,7 +33,6 @@ #include "arrow/testing/matchers.h" #include "arrow/testing/random.h" #include "arrow/util/checked_cast.h" -#include "arrow/util/make_unique.h" #include "arrow/util/thread_pool.h" using testing::UnorderedElementsAreArray; @@ -69,7 +69,7 @@ void CheckRunOutput(JoinType type, const BatchesWithSchema& l_batches, const std::vector& left_keys, const std::vector& right_keys, const BatchesWithSchema& exp_batches, bool parallel = false) { - auto exec_ctx = arrow::internal::make_unique( + auto exec_ctx = std::make_unique( default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); @@ -890,7 +890,7 @@ Result> HashJoinWithExecPlan( const std::shared_ptr& output_schema, const std::vector>& l, const std::vector>& r, int num_batches_l, int num_batches_r) { - auto exec_ctx = arrow::internal::make_unique( + auto exec_ctx = std::make_unique( default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); ARROW_ASSIGN_OR_RAISE(auto plan, ExecPlan::Make(exec_ctx.get())); @@ -1009,7 +1009,7 @@ TEST(HashJoin, Random) { for (int test_id = 0; test_id < num_tests; ++test_id) { bool parallel = (rng.from_range(0, 1) == 1); bool disable_bloom_filter = (rng.from_range(0, 1) == 1); - auto exec_ctx = arrow::internal::make_unique( + auto exec_ctx = std::make_unique( default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); // Constraints @@ -1310,7 +1310,7 @@ void TestHashJoinDictionaryHelper( r_batches.batches.resize(0); } - auto exec_ctx = arrow::internal::make_unique( + auto exec_ctx = std::make_unique( default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); ASSERT_OK_AND_ASSIGN( @@ -1735,7 +1735,7 @@ TEST(HashJoin, DictNegative) { i == 3 ? datumSecondB : datumSecondA})); auto exec_ctx = - arrow::internal::make_unique(default_memory_pool(), nullptr); + std::make_unique(default_memory_pool(), nullptr); ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); ASSERT_OK_AND_ASSIGN( ExecNode * l_source, @@ -1912,7 +1912,7 @@ TEST(HashJoin, ExtensionTypesHashJoin) { TEST(HashJoin, CheckHashJoinNodeOptionsValidation) { auto exec_ctx = - arrow::internal::make_unique(default_memory_pool(), nullptr); + std::make_unique(default_memory_pool(), nullptr); ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); BatchesWithSchema input_left; @@ -1997,7 +1997,7 @@ TEST(HashJoin, ResidualFilter) { input_right.schema = schema({field("r1", int32()), field("r2", int32()), field("r_str", utf8())}); - auto exec_ctx = arrow::internal::make_unique( + auto exec_ctx = std::make_unique( default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); @@ -2074,7 +2074,7 @@ TEST(HashJoin, TrivialResidualFilter) { ])")}; input_right.schema = schema({field("r1", int32()), field("r_str", utf8())}); - auto exec_ctx = arrow::internal::make_unique( + auto exec_ctx = std::make_unique( default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); @@ -2214,7 +2214,7 @@ void TestSingleChainOfHashJoins(Random64Bit& rng) { for (bool bloom_filters : {false, true}) { bool kParallel = true; ARROW_SCOPED_TRACE(bloom_filters ? "bloom filtered" : "unfiltered"); - auto exec_ctx = arrow::internal::make_unique( + auto exec_ctx = std::make_unique( default_memory_pool(), kParallel ? arrow::internal::GetCpuThreadPool() : nullptr); ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); diff --git a/cpp/src/arrow/compute/exec/plan_test.cc b/cpp/src/arrow/compute/exec/plan_test.cc index 1dd071975eec1..9f417d46ed323 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -35,7 +35,6 @@ #include "arrow/testing/random.h" #include "arrow/util/async_generator.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/thread_pool.h" #include "arrow/util/vector.h" @@ -1317,7 +1316,7 @@ TEST(ExecPlanExecution, SelfInnerHashJoinSink) { auto input = MakeGroupableBatches(); - auto exec_ctx = arrow::internal::make_unique( + auto exec_ctx = std::make_unique( default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); @@ -1374,7 +1373,7 @@ TEST(ExecPlanExecution, SelfOuterHashJoinSink) { auto input = MakeGroupableBatches(); - auto exec_ctx = arrow::internal::make_unique( + auto exec_ctx = std::make_unique( default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); diff --git a/cpp/src/arrow/compute/exec/tpch_benchmark.cc b/cpp/src/arrow/compute/exec/tpch_benchmark.cc index 5aad5370b738c..30f69871983bf 100644 --- a/cpp/src/arrow/compute/exec/tpch_benchmark.cc +++ b/cpp/src/arrow/compute/exec/tpch_benchmark.cc @@ -22,7 +22,8 @@ #include "arrow/compute/exec/test_util.h" #include "arrow/compute/exec/tpch_node.h" #include "arrow/testing/future_util.h" -#include "arrow/util/make_unique.h" + +#include namespace arrow { namespace compute { diff --git a/cpp/src/arrow/compute/exec/tpch_node.cc b/cpp/src/arrow/compute/exec/tpch_node.cc index 0cd33313b0375..0fa713b66bae0 100644 --- a/cpp/src/arrow/compute/exec/tpch_node.cc +++ b/cpp/src/arrow/compute/exec/tpch_node.cc @@ -37,7 +37,6 @@ #include "arrow/util/future.h" #include "arrow/util/io_util.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/pcg_random.h" #include "arrow/util/unreachable.h" @@ -3477,7 +3476,7 @@ class TpchGenImpl : public TpchGen { template Result TpchGenImpl::CreateNode(const char* name, std::vector columns) { - std::unique_ptr generator = arrow::internal::make_unique(); + std::unique_ptr generator = std::make_unique(); RETURN_NOT_OK(generator->Init(std::move(columns), scale_factor_, batch_size_, kSeedDist(seed_rng_))); return plan_->EmplaceNode(plan_, name, std::move(generator)); @@ -3492,7 +3491,7 @@ Result TpchGenImpl::Part(std::vector columns) { part_and_part_supp_generator_ = std::make_shared(); } std::unique_ptr generator = - arrow::internal::make_unique(part_and_part_supp_generator_); + std::make_unique(part_and_part_supp_generator_); RETURN_NOT_OK(generator->Init(std::move(columns), scale_factor_, batch_size_, kSeedDist(seed_rng_))); return plan_->EmplaceNode(plan_, "Part", std::move(generator)); @@ -3503,7 +3502,7 @@ Result TpchGenImpl::PartSupp(std::vector columns) { part_and_part_supp_generator_ = std::make_shared(); } std::unique_ptr generator = - arrow::internal::make_unique(part_and_part_supp_generator_); + std::make_unique(part_and_part_supp_generator_); RETURN_NOT_OK(generator->Init(std::move(columns), scale_factor_, batch_size_, kSeedDist(seed_rng_))); return plan_->EmplaceNode(plan_, "PartSupp", std::move(generator)); @@ -3518,7 +3517,7 @@ Result TpchGenImpl::Orders(std::vector columns) { orders_and_line_item_generator_ = std::make_shared(); } std::unique_ptr generator = - arrow::internal::make_unique(orders_and_line_item_generator_); + std::make_unique(orders_and_line_item_generator_); RETURN_NOT_OK(generator->Init(std::move(columns), scale_factor_, batch_size_, kSeedDist(seed_rng_))); return plan_->EmplaceNode(plan_, "Orders", std::move(generator)); @@ -3529,7 +3528,7 @@ Result TpchGenImpl::Lineitem(std::vector columns) { orders_and_line_item_generator_ = std::make_shared(); } std::unique_ptr generator = - arrow::internal::make_unique(orders_and_line_item_generator_); + std::make_unique(orders_and_line_item_generator_); RETURN_NOT_OK(generator->Init(std::move(columns), scale_factor_, batch_size_, kSeedDist(seed_rng_))); return plan_->EmplaceNode(plan_, "Lineitem", std::move(generator)); diff --git a/cpp/src/arrow/compute/exec/tpch_node_test.cc b/cpp/src/arrow/compute/exec/tpch_node_test.cc index dbc5b341d6007..bccfdfc3bc85d 100644 --- a/cpp/src/arrow/compute/exec/tpch_node_test.cc +++ b/cpp/src/arrow/compute/exec/tpch_node_test.cc @@ -18,6 +18,7 @@ #include #include +#include #include #include #include @@ -32,7 +33,6 @@ #include "arrow/testing/matchers.h" #include "arrow/testing/random.h" #include "arrow/util/checked_cast.h" -#include "arrow/util/make_unique.h" #include "arrow/util/pcg_random.h" #include "arrow/util/string.h" #include "arrow/util/thread_pool.h" diff --git a/cpp/src/arrow/compute/exec_test.cc b/cpp/src/arrow/compute/exec_test.cc index c31309da931ea..b1fa4e3846c70 100644 --- a/cpp/src/arrow/compute/exec_test.cc +++ b/cpp/src/arrow/compute/exec_test.cc @@ -43,7 +43,6 @@ #include "arrow/util/checked_cast.h" #include "arrow/util/cpu_info.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" namespace arrow { @@ -883,7 +882,7 @@ class ExampleOptionsType : public FunctionOptionsType { } std::unique_ptr Copy(const FunctionOptions& options) const override { const auto& opts = static_cast(options); - return arrow::internal::make_unique(opts.value); + return std::make_unique(opts.value); } }; ExampleOptions::ExampleOptions(std::shared_ptr value) diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic.cc b/cpp/src/arrow/compute/kernels/aggregate_basic.cc index ce8b7e867ec06..8e03a7b776f28 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_basic.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_basic.cc @@ -22,7 +22,8 @@ #include "arrow/compute/kernels/util_internal.h" #include "arrow/util/cpu_info.h" #include "arrow/util/hashing.h" -#include "arrow/util/make_unique.h" + +#include namespace arrow { namespace compute { @@ -119,7 +120,7 @@ struct CountImpl : public ScalarAggregator { Result> CountInit(KernelContext*, const KernelInitArgs& args) { - return ::arrow::internal::make_unique( + return std::make_unique( static_cast(*args.options)); } @@ -194,7 +195,7 @@ struct CountDistinctImpl : public ScalarAggregator { template Result> CountDistinctInit(KernelContext* ctx, const KernelInitArgs& args) { - return ::arrow::internal::make_unique>( + return std::make_unique>( ctx->memory_pool(), static_cast(*args.options)); } @@ -516,7 +517,7 @@ struct BooleanAnyImpl : public ScalarAggregator { Result> AnyInit(KernelContext*, const KernelInitArgs& args) { const ScalarAggregateOptions options = static_cast(*args.options); - return ::arrow::internal::make_unique( + return std::make_unique( static_cast(*args.options)); } @@ -586,7 +587,7 @@ struct BooleanAllImpl : public ScalarAggregator { }; Result> AllInit(KernelContext*, const KernelInitArgs& args) { - return ::arrow::internal::make_unique( + return std::make_unique( static_cast(*args.options)); } diff --git a/cpp/src/arrow/compute/kernels/codegen_internal.h b/cpp/src/arrow/compute/kernels/codegen_internal.h index b0001832174c7..65b56473fdad1 100644 --- a/cpp/src/arrow/compute/kernels/codegen_internal.h +++ b/cpp/src/arrow/compute/kernels/codegen_internal.h @@ -47,7 +47,6 @@ #include "arrow/util/decimal.h" #include "arrow/util/logging.h" #include "arrow/util/macros.h" -#include "arrow/util/make_unique.h" #include "arrow/visit_data_inline.h" namespace arrow { @@ -75,7 +74,7 @@ struct OptionsWrapper : public KernelState { static Result> Init(KernelContext* ctx, const KernelInitArgs& args) { if (auto options = static_cast(args.options)) { - return ::arrow::internal::make_unique(*options); + return std::make_unique(*options); } return Status::Invalid( @@ -101,7 +100,7 @@ struct KernelStateFromFunctionOptions : public KernelState { static Result> Init(KernelContext* ctx, const KernelInitArgs& args) { if (auto options = static_cast(args.options)) { - return ::arrow::internal::make_unique(ctx, + return std::make_unique(ctx, *options); } diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index f947cc732f77a..fe2b4af2059b1 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -49,7 +49,6 @@ #include "arrow/util/cpu_info.h" #include "arrow/util/int128_internal.h" #include "arrow/util/int_util_overflow.h" -#include "arrow/util/make_unique.h" #include "arrow/util/task_group.h" #include "arrow/util/tdigest.h" #include "arrow/util/thread_pool.h" @@ -84,7 +83,7 @@ struct GroupedAggregator : KernelState { template Result> HashAggregateInit(KernelContext* ctx, const KernelInitArgs& args) { - auto impl = ::arrow::internal::make_unique(); + auto impl = std::make_unique(); RETURN_NOT_OK(impl->Init(ctx->exec_context(), args)); return std::move(impl); } @@ -972,7 +971,7 @@ struct GroupedVarStdImpl : public GroupedAggregator { template Result> VarStdInit(KernelContext* ctx, const KernelInitArgs& args) { - auto impl = ::arrow::internal::make_unique>(); + auto impl = std::make_unique>(); impl->result_type_ = result_type; RETURN_NOT_OK(impl->Init(ctx->exec_context(), args)); return std::move(impl); diff --git a/cpp/src/arrow/compute/kernels/row_encoder.cc b/cpp/src/arrow/compute/kernels/row_encoder.cc index a38fa1db2059a..f553708cca5a3 100644 --- a/cpp/src/arrow/compute/kernels/row_encoder.cc +++ b/cpp/src/arrow/compute/kernels/row_encoder.cc @@ -19,7 +19,8 @@ #include "arrow/util/bitmap_writer.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" + +#include namespace arrow { diff --git a/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc b/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc index 984c3b56538d6..d18946599be89 100644 --- a/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc +++ b/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc @@ -655,7 +655,7 @@ struct RoundOptionsWrapper : public OptionsWrapper { static Result> Init(KernelContext* ctx, const KernelInitArgs& args) { if (auto options = static_cast(args.options)) { - return ::arrow::internal::make_unique(*options); + return std::make_unique(*options); } return Status::Invalid( "Attempted to initialize KernelState from null FunctionOptions"); @@ -700,10 +700,10 @@ struct RoundOptionsWrapper // Create a new option object if the rounding multiple was casted. auto new_options = OptionsType(casted_multiple.scalar(), options->round_mode); - return ::arrow::internal::make_unique(new_options); + return std::make_unique(new_options); } - return ::arrow::internal::make_unique(*options); + return std::make_unique(*options); } }; diff --git a/cpp/src/arrow/compute/kernels/scalar_string_ascii.cc b/cpp/src/arrow/compute/kernels/scalar_string_ascii.cc index db5eca79d96cd..4a6c95ece6d21 100644 --- a/cpp/src/arrow/compute/kernels/scalar_string_ascii.cc +++ b/cpp/src/arrow/compute/kernels/scalar_string_ascii.cc @@ -1222,7 +1222,7 @@ struct PlainSubstringMatcher { const MatchSubstringOptions& options) { // Should be handled by partial template specialization below DCHECK(!options.ignore_case); - return ::arrow::internal::make_unique(options); + return std::make_unique(options); } explicit PlainSubstringMatcher(const MatchSubstringOptions& options) @@ -1275,7 +1275,7 @@ struct PlainStartsWithMatcher { const MatchSubstringOptions& options) { // Should be handled by partial template specialization below DCHECK(!options.ignore_case); - return ::arrow::internal::make_unique(options); + return std::make_unique(options); } bool Match(std::string_view current) const { @@ -1293,7 +1293,7 @@ struct PlainEndsWithMatcher { const MatchSubstringOptions& options) { // Should be handled by partial template specialization below DCHECK(!options.ignore_case); - return ::arrow::internal::make_unique(options); + return std::make_unique(options); } bool Match(std::string_view current) const { @@ -1309,7 +1309,7 @@ struct RegexSubstringMatcher { static Result> Make( const MatchSubstringOptions& options, bool is_utf8 = true, bool literal = false) { auto matcher = - ::arrow::internal::make_unique(options, is_utf8, literal); + std::make_unique(options, is_utf8, literal); RETURN_NOT_OK(RegexStatus(matcher->regex_match_)); return std::move(matcher); } @@ -1974,7 +1974,7 @@ struct PlainSubstringReplacer { static Result> Make( const ReplaceSubstringOptions& options) { - return arrow::internal::make_unique(options); + return std::make_unique(options); } explicit PlainSubstringReplacer(const ReplaceSubstringOptions& options) @@ -2023,7 +2023,7 @@ struct RegexSubstringReplacer { static Result> Make( const ReplaceSubstringOptions& options) { - auto replacer = arrow::internal::make_unique(options); + auto replacer = std::make_unique(options); RETURN_NOT_OK(RegexStatus(replacer->regex_find_)); RETURN_NOT_OK(RegexStatus(replacer->regex_replacement_)); @@ -2581,7 +2581,7 @@ struct SplitRegexFinder : public StringSplitFinderBase { pattern.reserve(options.pattern.size() + 2); pattern += options.pattern; pattern += ')'; - regex_split = arrow::internal::make_unique(pattern, MakeRE2Options()); + regex_split = std::make_unique(pattern, MakeRE2Options()); return RegexStatus(*regex_split); } diff --git a/cpp/src/arrow/compute/kernels/vector_cumulative_ops.cc b/cpp/src/arrow/compute/kernels/vector_cumulative_ops.cc index fb221aa9fe6c9..76bda200dadef 100644 --- a/cpp/src/arrow/compute/kernels/vector_cumulative_ops.cc +++ b/cpp/src/arrow/compute/kernels/vector_cumulative_ops.cc @@ -59,9 +59,9 @@ struct CumulativeOptionsWrapper : public OptionsWrapper { auto casted_start, Cast(Datum(start), args.inputs[0], CastOptions::Safe(), ctx->exec_context())); auto new_options = OptionsType(casted_start.scalar(), options->skip_nulls); - return ::arrow::internal::make_unique(new_options); + return std::make_unique(new_options); } - return ::arrow::internal::make_unique(*options); + return std::make_unique(*options); } }; diff --git a/cpp/src/arrow/compute/kernels/vector_hash.cc b/cpp/src/arrow/compute/kernels/vector_hash.cc index c294992d27f0e..c6e38e6f54e33 100644 --- a/cpp/src/arrow/compute/kernels/vector_hash.cc +++ b/cpp/src/arrow/compute/kernels/vector_hash.cc @@ -16,6 +16,7 @@ // under the License. #include +#include #include #include "arrow/array/array_base.h" @@ -29,7 +30,6 @@ #include "arrow/compute/kernels/common.h" #include "arrow/result.h" #include "arrow/util/hashing.h" -#include "arrow/util/make_unique.h" namespace arrow { @@ -524,7 +524,7 @@ template Result> HashInitImpl(KernelContext* ctx, const KernelInitArgs& args) { using HashKernelType = typename HashKernelTraits::HashKernel; - auto result = ::arrow::internal::make_unique( + auto result = std::make_unique( args.inputs[0].GetSharedPtr(), args.options, ctx->memory_pool()); RETURN_NOT_OK(result->Reset()); return std::move(result); @@ -614,7 +614,7 @@ Result> DictionaryHashInit(KernelContext* ctx, break; } RETURN_NOT_OK(indices_hasher); - return ::arrow::internal::make_unique( + return std::make_unique( std::move(indices_hasher.ValueOrDie()), dict_type.value_type()); } diff --git a/cpp/src/arrow/compute/kernels/vector_replace_test.cc b/cpp/src/arrow/compute/kernels/vector_replace_test.cc index b83b6973313c9..39d6ff1d470f1 100644 --- a/cpp/src/arrow/compute/kernels/vector_replace_test.cc +++ b/cpp/src/arrow/compute/kernels/vector_replace_test.cc @@ -26,7 +26,8 @@ #include "arrow/testing/gtest_util.h" #include "arrow/util/checked_cast.h" #include "arrow/util/key_value_metadata.h" -#include "arrow/util/make_unique.h" + +#include namespace arrow { namespace compute { @@ -157,7 +158,7 @@ class TestReplaceKernel : public ::testing::Test { const typename TypeTraits::ArrayType& array, const BooleanArray& mask, const typename TypeTraits::ArrayType& replacements) { auto length = array.length(); - auto builder = arrow::internal::make_unique::BuilderType>( + auto builder = std::make_unique::BuilderType>( default_type_instance(), default_memory_pool()); int64_t replacement_offset = 0; for (int64_t i = 0; i < length; ++i) { diff --git a/cpp/src/arrow/compute/registry_test.cc b/cpp/src/arrow/compute/registry_test.cc index 5517f3090d91e..7fee136de7a0b 100644 --- a/cpp/src/arrow/compute/registry_test.cc +++ b/cpp/src/arrow/compute/registry_test.cc @@ -27,7 +27,6 @@ #include "arrow/status.h" #include "arrow/testing/gtest_util.h" #include "arrow/util/macros.h" -#include "arrow/util/make_unique.h" namespace arrow { namespace compute { @@ -184,7 +183,7 @@ class ExampleOptionsType : public FunctionOptionsType { } std::unique_ptr Copy(const FunctionOptions& options) const override { const auto& opts = static_cast&>(options); - return arrow::internal::make_unique>(opts.value); + return std::make_unique>(opts.value); } }; template diff --git a/cpp/src/arrow/compute/row/grouper.cc b/cpp/src/arrow/compute/row/grouper.cc index d6d00c2cce62f..20c2e5e8d1f9f 100644 --- a/cpp/src/arrow/compute/row/grouper.cc +++ b/cpp/src/arrow/compute/row/grouper.cc @@ -17,6 +17,7 @@ #include "arrow/compute/row/grouper.h" +#include #include #include "arrow/compute/exec/key_hash.h" @@ -33,7 +34,6 @@ #include "arrow/util/checked_cast.h" #include "arrow/util/cpu_info.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/task_group.h" namespace arrow { @@ -47,7 +47,7 @@ namespace { struct GrouperImpl : Grouper { static Result> Make( const std::vector& key_types, ExecContext* ctx) { - auto impl = ::arrow::internal::make_unique(); + auto impl = std::make_unique(); impl->encoders_.resize(key_types.size()); impl->ctx_ = ctx; @@ -58,38 +58,38 @@ struct GrouperImpl : Grouper { if (key->id() == Type::BOOL) { impl->encoders_[i] = - ::arrow::internal::make_unique(); + std::make_unique(); continue; } if (key->id() == Type::DICTIONARY) { impl->encoders_[i] = - ::arrow::internal::make_unique( + std::make_unique( key, ctx->memory_pool()); continue; } if (is_fixed_width(key->id())) { impl->encoders_[i] = - ::arrow::internal::make_unique(key); + std::make_unique(key); continue; } if (is_binary_like(key->id())) { impl->encoders_[i] = - ::arrow::internal::make_unique>( + std::make_unique>( key); continue; } if (is_large_binary_like(key->id())) { - impl->encoders_[i] = ::arrow::internal::make_unique< + impl->encoders_[i] = std::make_unique< internal::VarLengthKeyEncoder>(key); continue; } if (key->id() == Type::NA) { - impl->encoders_[i] = ::arrow::internal::make_unique(); + impl->encoders_[i] = std::make_unique(); continue; } @@ -202,7 +202,7 @@ struct GrouperFastImpl : Grouper { static Result> Make( const std::vector& keys, ExecContext* ctx) { - auto impl = ::arrow::internal::make_unique(); + auto impl = std::make_unique(); impl->ctx_ = ctx; RETURN_NOT_OK(impl->temp_stack_.Init(ctx->memory_pool(), 64 * minibatch_size_max_)); diff --git a/cpp/src/arrow/csv/chunker.cc b/cpp/src/arrow/csv/chunker.cc index bc1b69cb8ae89..c71b4ef234dd2 100644 --- a/cpp/src/arrow/csv/chunker.cc +++ b/cpp/src/arrow/csv/chunker.cc @@ -26,7 +26,6 @@ #include "arrow/csv/lexing_internal.h" #include "arrow/status.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" namespace arrow { namespace csv { @@ -395,7 +394,7 @@ std::unique_ptr MakeChunker(const ParseOptions& options) { } } } - return ::arrow::internal::make_unique(std::move(delimiter)); + return std::make_unique(std::move(delimiter)); } } // namespace csv diff --git a/cpp/src/arrow/csv/writer.cc b/cpp/src/arrow/csv/writer.cc index bb8d555a789f5..793a701a7101a 100644 --- a/cpp/src/arrow/csv/writer.cc +++ b/cpp/src/arrow/csv/writer.cc @@ -26,10 +26,11 @@ #include "arrow/stl_allocator.h" #include "arrow/util/iterator.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/visit_data_inline.h" #include "arrow/visit_type_inline.h" +#include + #if defined(ARROW_HAVE_NEON) || defined(ARROW_HAVE_SSE4_2) #include #endif diff --git a/cpp/src/arrow/dataset/dataset.cc b/cpp/src/arrow/dataset/dataset.cc index eb307681e916a..57ee6726cd1a0 100644 --- a/cpp/src/arrow/dataset/dataset.cc +++ b/cpp/src/arrow/dataset/dataset.cc @@ -26,7 +26,6 @@ #include "arrow/util/bit_util.h" #include "arrow/util/iterator.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/thread_pool.h" namespace arrow { diff --git a/cpp/src/arrow/dataset/dataset_writer.cc b/cpp/src/arrow/dataset/dataset_writer.cc index d8e00054e1c0b..8fcbea656d932 100644 --- a/cpp/src/arrow/dataset/dataset_writer.cc +++ b/cpp/src/arrow/dataset/dataset_writer.cc @@ -18,6 +18,7 @@ #include "arrow/dataset/dataset_writer.h" #include +#include #include #include @@ -27,7 +28,6 @@ #include "arrow/table.h" #include "arrow/util/future.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/map.h" #include "arrow/util/string.h" @@ -186,7 +186,7 @@ class DatasetWriterFileQueue { std::shared_ptr batch; }; scheduler_->AddTask( - ::arrow::internal::make_unique(this, std::move(batch))); + std::make_unique(this, std::move(batch))); } Result PopAndDeliverStagedBatch() { @@ -331,7 +331,7 @@ class DatasetWriterDirectoryQueue { } Result OpenFileQueue(const std::string& filename) { - auto file_queue = ::arrow::internal::make_unique( + auto file_queue = std::make_unique( schema_, write_options_, writer_state_); DatasetWriterFileQueue* file_queue_view = file_queue.get(); std::unique_ptr throttle = @@ -384,7 +384,7 @@ class DatasetWriterDirectoryQueue { const FileSystemDatasetWriteOptions& write_options, DatasetWriterState* writer_state, std::shared_ptr schema, std::string directory, std::string prefix) { - auto dir_queue = ::arrow::internal::make_unique( + auto dir_queue = std::make_unique( scheduler, std::move(directory), std::move(prefix), std::move(schema), write_options, writer_state); dir_queue->PrepareDirectory(); @@ -585,7 +585,7 @@ class DatasetWriter::DatasetWriterImpl { DatasetWriter::DatasetWriter(FileSystemDatasetWriteOptions write_options, util::AsyncTaskScheduler* scheduler, uint64_t max_rows_queued) - : impl_(::arrow::internal::make_unique( + : impl_(std::make_unique( std::move(write_options), scheduler, max_rows_queued)) {} Result> DatasetWriter::Make( diff --git a/cpp/src/arrow/dataset/file_base.cc b/cpp/src/arrow/dataset/file_base.cc index ecf5d106d2e53..98dccead986dd 100644 --- a/cpp/src/arrow/dataset/file_base.cc +++ b/cpp/src/arrow/dataset/file_base.cc @@ -20,6 +20,7 @@ #include #include +#include #include #include #include @@ -40,7 +41,6 @@ #include "arrow/util/compression.h" #include "arrow/util/iterator.h" #include "arrow/util/macros.h" -#include "arrow/util/make_unique.h" #include "arrow/util/map.h" #include "arrow/util/string.h" #include "arrow/util/task_group.h" diff --git a/cpp/src/arrow/dataset/file_csv_test.cc b/cpp/src/arrow/dataset/file_csv_test.cc index 99c6494f194ea..8062209eb077e 100644 --- a/cpp/src/arrow/dataset/file_csv_test.cc +++ b/cpp/src/arrow/dataset/file_csv_test.cc @@ -65,7 +65,7 @@ class TestCsvFileFormat : public FileFormatFixtureMixin, std::unique_ptr GetFileSource(std::string csv) { if (GetCompression() == Compression::UNCOMPRESSED) { - return ::arrow::internal::make_unique( + return std::make_unique( Buffer::FromString(std::move(csv))); } std::string path = "test.csv"; @@ -94,7 +94,7 @@ class TestCsvFileFormat : public FileFormatFixtureMixin, ARROW_EXPECT_OK(stream->Write(csv)); ARROW_EXPECT_OK(stream->Close()); EXPECT_OK_AND_ASSIGN(auto info, fs->GetFileInfo(path)); - return ::arrow::internal::make_unique(info, fs, GetCompression()); + return std::make_unique(info, fs, GetCompression()); } RecordBatchIterator Batches(Fragment* fragment) { diff --git a/cpp/src/arrow/dataset/partition.cc b/cpp/src/arrow/dataset/partition.cc index 48594336878a4..1a0c5c8fcf8e6 100644 --- a/cpp/src/arrow/dataset/partition.cc +++ b/cpp/src/arrow/dataset/partition.cc @@ -37,7 +37,6 @@ #include "arrow/scalar.h" #include "arrow/util/int_util_overflow.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/uri.h" #include "arrow/util/utf8.h" @@ -562,7 +561,7 @@ class KeyValuePartitioningFactory : public PartitioningFactory { } std::unique_ptr MakeMemo() { - return ::arrow::internal::make_unique(default_memory_pool(), + return std::make_unique(default_memory_pool(), utf8()); } diff --git a/cpp/src/arrow/dataset/test_util.h b/cpp/src/arrow/dataset/test_util.h index e17afd25a34fc..0a123892c4863 100644 --- a/cpp/src/arrow/dataset/test_util.h +++ b/cpp/src/arrow/dataset/test_util.h @@ -51,7 +51,6 @@ #include "arrow/util/io_util.h" #include "arrow/util/iterator.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/thread_pool.h" namespace arrow { @@ -92,7 +91,7 @@ void AssertDatasetHasSchema(std::shared_ptr ds, std::shared_ptr class FileSourceFixtureMixin : public ::testing::Test { public: std::unique_ptr GetSource(std::shared_ptr buffer) { - return ::arrow::internal::make_unique(std::move(buffer)); + return std::make_unique(std::move(buffer)); } }; @@ -114,7 +113,7 @@ class GeneratedRecordBatch : public RecordBatchReader { template std::unique_ptr> MakeGeneratedRecordBatch( std::shared_ptr schema, Gen&& gen) { - return ::arrow::internal::make_unique>( + return std::make_unique>( schema, std::forward(gen)); } diff --git a/cpp/src/arrow/engine/substrait/expression_internal.cc b/cpp/src/arrow/engine/substrait/expression_internal.cc index 6f181ac0218c4..4088da70898b9 100644 --- a/cpp/src/arrow/engine/substrait/expression_internal.cc +++ b/cpp/src/arrow/engine/substrait/expression_internal.cc @@ -20,6 +20,7 @@ #include "arrow/engine/substrait/expression_internal.h" #include +#include #include "arrow/builder.h" #include "arrow/compute/exec/expression.h" @@ -28,7 +29,6 @@ #include "arrow/engine/substrait/type_internal.h" #include "arrow/result.h" #include "arrow/status.h" -#include "arrow/util/make_unique.h" #include "arrow/visit_scalar_inline.h" namespace arrow { @@ -38,7 +38,7 @@ using internal::checked_cast; namespace engine { namespace internal { -using ::arrow::internal::make_unique; +using std::make_unique; } // namespace internal Status DecodeArg(const substrait::FunctionArgument& arg, uint32_t idx, @@ -608,7 +608,7 @@ struct ScalarToProtoImpl { Status Visit(const DayTimeIntervalScalar& s) { return NotImplemented(s); } Status Visit(const Decimal128Scalar& s) { - auto decimal = internal::make_unique(); + auto decimal = std::make_unique(); auto decimal_type = checked_cast(s.type.get()); decimal->set_precision(decimal_type->precision()); @@ -685,7 +685,7 @@ struct ScalarToProtoImpl { key_values->Reserve(static_cast(kv_arr.length())); for (int64_t i = 0; i < s.value->length(); ++i) { - auto kv = internal::make_unique(); + auto kv = std::make_unique(); ARROW_ASSIGN_OR_RAISE(Datum key_scalar, kv_arr.field(0)->GetScalar(i)); ARROW_ASSIGN_OR_RAISE(auto key, ToProto(key_scalar, ext_set_, conversion_options_)); @@ -714,7 +714,7 @@ struct ScalarToProtoImpl { } if (auto length = UnwrapVarChar(*s.type)) { - auto var_char = internal::make_unique(); + auto var_char = std::make_unique(); var_char->set_length(*length); var_char->set_value(checked_cast(*s.value).value->ToString()); @@ -729,7 +729,7 @@ struct ScalarToProtoImpl { }; if (UnwrapIntervalYear(*s.type)) { - auto interval_year = internal::make_unique(); + auto interval_year = std::make_unique(); interval_year->set_years(GetPairOfInts().first); interval_year->set_months(GetPairOfInts().second); @@ -738,7 +738,7 @@ struct ScalarToProtoImpl { } if (UnwrapIntervalDay(*s.type)) { - auto interval_day = internal::make_unique(); + auto interval_day = std::make_unique(); interval_day->set_days(GetPairOfInts().first); interval_day->set_seconds(GetPairOfInts().second); @@ -777,7 +777,7 @@ Result> ToProto( " as a substrait::Expression::Literal"); } - auto out = internal::make_unique(); + auto out = std::make_unique(); if (datum.scalar()->is_valid) { RETURN_NOT_OK( @@ -849,7 +849,7 @@ static Result> MakeDirectReference( } } - auto selection = internal::make_unique(); + auto selection = std::make_unique(); selection->set_allocated_direct_reference(ref_segment.release()); if (expr && expr->rex_type_case() != substrait::Expression::REX_TYPE_NOT_SET) { @@ -859,7 +859,7 @@ static Result> MakeDirectReference( new substrait::Expression::FieldReference::RootReference()); } - auto out = internal::make_unique(); + auto out = std::make_unique(); out->set_allocated_selection(selection.release()); return std::move(out); } @@ -869,10 +869,10 @@ static Result> MakeDirectReference( static Result> MakeStructFieldReference( std::unique_ptr&& expr, int field) { auto struct_field = - internal::make_unique(); + std::make_unique(); struct_field->set_field(field); - auto ref_segment = internal::make_unique(); + auto ref_segment = std::make_unique(); ref_segment->set_allocated_struct_field(struct_field.release()); return MakeDirectReference(std::move(expr), std::move(ref_segment)); @@ -882,10 +882,10 @@ static Result> MakeStructFieldReference( static Result> MakeListElementReference( std::unique_ptr&& expr, int offset) { auto list_element = - internal::make_unique(); + std::make_unique(); list_element->set_offset(offset); - auto ref_segment = internal::make_unique(); + auto ref_segment = std::make_unique(); ref_segment->set_allocated_list_element(list_element.release()); return MakeDirectReference(std::move(expr), std::move(ref_segment)); @@ -895,7 +895,7 @@ Result> EncodeSubstraitCa const SubstraitCall& call, ExtensionSet* ext_set, const ConversionOptions& conversion_options) { ARROW_ASSIGN_OR_RAISE(uint32_t anchor, ext_set->EncodeFunction(call.id())); - auto scalar_fn = internal::make_unique(); + auto scalar_fn = std::make_unique(); scalar_fn->set_function_reference(anchor); ARROW_ASSIGN_OR_RAISE( std::unique_ptr output_type, @@ -905,7 +905,7 @@ Result> EncodeSubstraitCa for (uint32_t i = 0; i < call.size(); i++) { substrait::FunctionArgument* arg = scalar_fn->add_arguments(); if (call.HasEnumArg(i)) { - auto enum_val = internal::make_unique(); + auto enum_val = std::make_unique(); ARROW_ASSIGN_OR_RAISE(std::optional enum_arg, call.GetEnumArg(i)); if (enum_arg) { enum_val->set_specified(std::string(*enum_arg)); @@ -933,7 +933,7 @@ Result> ToProto( return Status::Invalid("ToProto requires a bound Expression"); } - auto out = internal::make_unique(); + auto out = std::make_unique(); if (auto datum = expr.literal()) { ARROW_ASSIGN_OR_RAISE(auto literal, ToProto(*datum, ext_set, conversion_options)); @@ -958,7 +958,7 @@ Result> ToProto( auto conditions = call->arguments[0].call(); if (conditions && conditions->function_name == "make_struct") { // catch the special case of calls convertible to IfThen - auto if_then_ = internal::make_unique(); + auto if_then_ = std::make_unique(); // don't try to convert argument 0 of the case_when; we have to convert the elements // of make_struct individually @@ -972,7 +972,7 @@ Result> ToProto( for (size_t i = 0; i < conditions->arguments.size(); ++i) { ARROW_ASSIGN_OR_RAISE(auto cond_substrait, ToProto(conditions->arguments[i], ext_set, conversion_options)); - auto clause = internal::make_unique(); + auto clause = std::make_unique(); clause->set_allocated_if_(cond_substrait.release()); clause->set_allocated_then(arguments[i].release()); if_then_->mutable_ifs()->AddAllocated(clause.release()); @@ -1019,11 +1019,11 @@ Result> ToProto( if (call->function_name == "if_else") { // catch the special case of calls convertible to IfThen - auto if_clause = internal::make_unique(); + auto if_clause = std::make_unique(); if_clause->set_allocated_if_(arguments[0].release()); if_clause->set_allocated_then(arguments[1].release()); - auto if_then = internal::make_unique(); + auto if_then = std::make_unique(); if_then->mutable_ifs()->AddAllocated(if_clause.release()); if_then->set_allocated_else_(arguments[2].release()); diff --git a/cpp/src/arrow/engine/substrait/extension_set.cc b/cpp/src/arrow/engine/substrait/extension_set.cc index 2f7c85c9d5ccf..6ffe534c19697 100644 --- a/cpp/src/arrow/engine/substrait/extension_set.cc +++ b/cpp/src/arrow/engine/substrait/extension_set.cc @@ -18,13 +18,13 @@ #include "arrow/engine/substrait/extension_set.h" #include +#include #include #include #include "arrow/engine/substrait/expression_internal.h" #include "arrow/util/hash_util.h" #include "arrow/util/hashing.h" -#include "arrow/util/make_unique.h" namespace arrow { namespace engine { @@ -123,7 +123,7 @@ class IdStorageImpl : public IdStorage { }; std::unique_ptr IdStorage::Make() { - return ::arrow::internal::make_unique(); + return std::make_unique(); } Result> SubstraitCall::GetEnumArg(uint32_t index) const { diff --git a/cpp/src/arrow/engine/substrait/plan_internal.cc b/cpp/src/arrow/engine/substrait/plan_internal.cc index bd30f043a1b2c..f1072ca1a5fd3 100644 --- a/cpp/src/arrow/engine/substrait/plan_internal.cc +++ b/cpp/src/arrow/engine/substrait/plan_internal.cc @@ -22,9 +22,9 @@ #include "arrow/result.h" #include "arrow/util/hashing.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/unreachable.h" +#include #include namespace arrow { @@ -34,7 +34,7 @@ using internal::checked_cast; namespace engine { namespace internal { -using ::arrow::internal::make_unique; +using std::make_unique; } // namespace internal Status AddExtensionSetToPlan(const ExtensionSet& ext_set, substrait::Plan* plan) { @@ -48,7 +48,7 @@ Status AddExtensionSetToPlan(const ExtensionSet& ext_set, substrait::Plan* plan) auto uri = ext_set.uris().at(anchor); if (uri.empty()) continue; - auto ext_uri = internal::make_unique(); + auto ext_uri = std::make_unique(); ext_uri->set_uri(std::string(uri)); ext_uri->set_extension_uri_anchor(anchor); uris->AddAllocated(ext_uri.release()); @@ -65,9 +65,9 @@ Status AddExtensionSetToPlan(const ExtensionSet& ext_set, substrait::Plan* plan) ARROW_ASSIGN_OR_RAISE(auto type_record, ext_set.DecodeType(anchor)); if (type_record.id.empty()) continue; - auto ext_decl = internal::make_unique(); + auto ext_decl = std::make_unique(); - auto type = internal::make_unique(); + auto type = std::make_unique(); type->set_extension_uri_reference(map[type_record.id.uri]); type->set_type_anchor(anchor); type->set_name(std::string(type_record.id.name)); @@ -78,12 +78,12 @@ Status AddExtensionSetToPlan(const ExtensionSet& ext_set, substrait::Plan* plan) for (uint32_t anchor = 0; anchor < ext_set.num_functions(); ++anchor) { ARROW_ASSIGN_OR_RAISE(Id function_id, ext_set.DecodeFunction(anchor)); - auto fn = internal::make_unique(); + auto fn = std::make_unique(); fn->set_extension_uri_reference(map[function_id.uri]); fn->set_function_anchor(anchor); fn->set_name(std::string(function_id.name)); - auto ext_decl = internal::make_unique(); + auto ext_decl = std::make_unique(); ext_decl->set_allocated_extension_function(fn.release()); extensions->AddAllocated(ext_decl.release()); } @@ -138,9 +138,9 @@ Result GetExtensionSetFromPlan(const substrait::Plan& plan, Result> PlanToProto( const compute::Declaration& declr, ExtensionSet* ext_set, const ConversionOptions& conversion_options) { - auto subs_plan = internal::make_unique(); - auto plan_rel = internal::make_unique(); - auto rel_root = internal::make_unique(); + auto subs_plan = std::make_unique(); + auto plan_rel = std::make_unique(); + auto rel_root = std::make_unique(); ARROW_ASSIGN_OR_RAISE(auto rel, ToProto(declr, ext_set, conversion_options)); rel_root->set_allocated_input(rel.release()); plan_rel->set_allocated_root(rel_root.release()); diff --git a/cpp/src/arrow/engine/substrait/relation_internal.cc b/cpp/src/arrow/engine/substrait/relation_internal.cc index c920a1a46d0d0..70e2e012b7edf 100644 --- a/cpp/src/arrow/engine/substrait/relation_internal.cc +++ b/cpp/src/arrow/engine/substrait/relation_internal.cc @@ -30,14 +30,15 @@ #include "arrow/filesystem/path_util.h" #include "arrow/filesystem/util_internal.h" #include "arrow/util/checked_cast.h" -#include "arrow/util/make_unique.h" #include "arrow/util/string.h" #include "arrow/util/uri.h" +#include + namespace arrow { using internal::checked_cast; -using internal::make_unique; +using std::make_unique; using internal::StartsWith; using internal::UriFromAbsolutePath; diff --git a/cpp/src/arrow/engine/substrait/test_plan_builder.cc b/cpp/src/arrow/engine/substrait/test_plan_builder.cc index 3bd373ae5fa56..79e7fd40537d2 100644 --- a/cpp/src/arrow/engine/substrait/test_plan_builder.cc +++ b/cpp/src/arrow/engine/substrait/test_plan_builder.cc @@ -18,12 +18,12 @@ #include "arrow/engine/substrait/test_plan_builder.h" #include +#include #include "arrow/compute/exec/exec_plan.h" #include "arrow/engine/substrait/plan_internal.h" #include "arrow/engine/substrait/type_internal.h" #include "arrow/util/macros.h" -#include "arrow/util/make_unique.h" #include "substrait/algebra.pb.h" #include "substrait/plan.pb.h" @@ -31,7 +31,7 @@ namespace arrow { -using internal::make_unique; +using std::make_unique; namespace engine { namespace internal { diff --git a/cpp/src/arrow/engine/substrait/type_internal.cc b/cpp/src/arrow/engine/substrait/type_internal.cc index a2b5445cdce1f..b85d06626272a 100644 --- a/cpp/src/arrow/engine/substrait/type_internal.cc +++ b/cpp/src/arrow/engine/substrait/type_internal.cc @@ -17,6 +17,7 @@ #include "arrow/engine/substrait/type_internal.h" +#include #include #include @@ -25,14 +26,13 @@ #include "arrow/status.h" #include "arrow/type.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/visit_type_inline.h" namespace arrow { namespace engine { namespace internal { -using ::arrow::internal::make_unique; +using std::make_unique; } // namespace internal namespace { @@ -381,7 +381,7 @@ struct DataTypeToProtoImpl { template Sub* SetWithThen(void (::substrait::Type::*set_allocated_sub)(Sub*)) { - auto sub = internal::make_unique(); + auto sub = std::make_unique(); sub->set_nullability(nullable_ ? ::substrait::Type::NULLABILITY_NULLABLE : ::substrait::Type::NULLABILITY_REQUIRED); @@ -398,7 +398,7 @@ struct DataTypeToProtoImpl { template Status EncodeUserDefined(const T& t) { ARROW_ASSIGN_OR_RAISE(auto anchor, ext_set_->EncodeType(t)); - auto user_defined = internal::make_unique<::substrait::Type::UserDefined>(); + auto user_defined = std::make_unique<::substrait::Type::UserDefined>(); user_defined->set_type_reference(anchor); user_defined->set_nullability(nullable_ ? ::substrait::Type::NULLABILITY_NULLABLE : ::substrait::Type::NULLABILITY_REQUIRED); @@ -422,7 +422,7 @@ struct DataTypeToProtoImpl { Result> ToProto( const DataType& type, bool nullable, ExtensionSet* ext_set, const ConversionOptions& conversion_options) { - auto out = internal::make_unique<::substrait::Type>(); + auto out = std::make_unique<::substrait::Type>(); RETURN_NOT_OK( (DataTypeToProtoImpl{out.get(), nullable, ext_set, conversion_options})(type)); return std::move(out); @@ -479,13 +479,13 @@ Result> ToProto( return Status::Invalid("::substrait::NamedStruct does not support schema metadata"); } - auto named_struct = internal::make_unique<::substrait::NamedStruct>(); + auto named_struct = std::make_unique<::substrait::NamedStruct>(); auto names = named_struct->mutable_names(); names->Reserve(schema.num_fields()); ToProtoGetDepthFirstNames(schema.fields(), names); - auto struct_ = internal::make_unique<::substrait::Type::Struct>(); + auto struct_ = std::make_unique<::substrait::Type::Struct>(); auto types = struct_->mutable_types(); types->Reserve(schema.num_fields()); diff --git a/cpp/src/arrow/filesystem/localfs_benchmark.cc b/cpp/src/arrow/filesystem/localfs_benchmark.cc index 3c4ded7e537da..1b43325efb8cd 100644 --- a/cpp/src/arrow/filesystem/localfs_benchmark.cc +++ b/cpp/src/arrow/filesystem/localfs_benchmark.cc @@ -29,13 +29,12 @@ #include "arrow/testing/random.h" #include "arrow/util/async_generator.h" #include "arrow/util/io_util.h" -#include "arrow/util/make_unique.h" namespace arrow { namespace fs { -using arrow::internal::make_unique; +using std::make_unique; using arrow::internal::TemporaryDir; /// Set up hierarchical directory structure to test asynchronous diff --git a/cpp/src/arrow/flight/client.cc b/cpp/src/arrow/flight/client.cc index c88ce4a08b293..c3ebda1d2d2c6 100644 --- a/cpp/src/arrow/flight/client.cc +++ b/cpp/src/arrow/flight/client.cc @@ -33,7 +33,6 @@ #include "arrow/status.h" #include "arrow/table.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/flight/client_auth.h" #include "arrow/flight/serialization_internal.h" @@ -634,7 +633,7 @@ arrow::Result> FlightClient::DoGet( std::unique_ptr remote_stream; RETURN_NOT_OK(transport_->DoGet(options, ticket, &remote_stream)); std::unique_ptr stream_reader = - arrow::internal::make_unique( + std::make_unique( std::move(remote_stream), options.read_options, options.stop_token, options.memory_manager); // Eagerly read the schema @@ -656,8 +655,8 @@ arrow::Result FlightClient::DoPut( RETURN_NOT_OK(transport_->DoPut(options, &remote_stream)); std::shared_ptr shared_stream = std::move(remote_stream); DoPutResult result; - result.reader = arrow::internal::make_unique(shared_stream); - result.writer = arrow::internal::make_unique( + result.reader = std::make_unique(shared_stream); + result.writer = std::make_unique( std::move(shared_stream), options.write_options, write_size_limit_bytes_, descriptor); RETURN_NOT_OK(result.writer->Begin(schema, options.write_options)); @@ -682,9 +681,9 @@ arrow::Result FlightClient::DoExchange( RETURN_NOT_OK(transport_->DoExchange(options, &remote_stream)); std::shared_ptr shared_stream = std::move(remote_stream); DoExchangeResult result; - result.reader = arrow::internal::make_unique( + result.reader = std::make_unique( shared_stream, options.read_options, options.stop_token, options.memory_manager); - auto stream_writer = arrow::internal::make_unique( + auto stream_writer = std::make_unique( std::move(shared_stream), options.write_options, write_size_limit_bytes_, descriptor); RETURN_NOT_OK(stream_writer->Begin()); diff --git a/cpp/src/arrow/flight/cookie_internal.cc b/cpp/src/arrow/flight/cookie_internal.cc index 37672d2ecd860..2f691f769bf60 100644 --- a/cpp/src/arrow/flight/cookie_internal.cc +++ b/cpp/src/arrow/flight/cookie_internal.cc @@ -23,11 +23,12 @@ #include "arrow/flight/client_auth.h" #include "arrow/flight/platform.h" #include "arrow/util/base64.h" -#include "arrow/util/make_unique.h" #include "arrow/util/string.h" #include "arrow/util/uri.h" #include "arrow/util/value_parsing.h" +#include + // Mingw-w64 defines strcasecmp in string.h #if defined(_WIN32) && !defined(strcasecmp) #define strcasecmp stricmp diff --git a/cpp/src/arrow/flight/flight_test.cc b/cpp/src/arrow/flight/flight_test.cc index 54597e5420368..561fa055e4a56 100644 --- a/cpp/src/arrow/flight/flight_test.cc +++ b/cpp/src/arrow/flight/flight_test.cc @@ -38,7 +38,6 @@ #include "arrow/testing/util.h" #include "arrow/util/base64.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #ifdef GRPCPP_GRPCPP_H #error "gRPC headers should not be in public API" @@ -608,7 +607,7 @@ class PropagatingClientMiddlewareFactory : public ClientMiddlewareFactory { public: void StartCall(const CallInfo& info, std::unique_ptr* middleware) { recorded_calls_.push_back(info.method); - *middleware = arrow::internal::make_unique( + *middleware = std::make_unique( &received_headers_, &recorded_status_); } @@ -1393,14 +1392,14 @@ TEST_F(TestBasicHeaderAuthMiddleware, InvalidCredentials) { RunInvalidClientAuth class ForeverFlightListing : public FlightListing { arrow::Result> Next() override { std::this_thread::sleep_for(std::chrono::milliseconds(100)); - return arrow::internal::make_unique(ExampleFlightInfo()[0]); + return std::make_unique(ExampleFlightInfo()[0]); } }; class ForeverResultStream : public ResultStream { arrow::Result> Next() override { std::this_thread::sleep_for(std::chrono::milliseconds(100)); - auto result = arrow::internal::make_unique(); + auto result = std::make_unique(); result->body = Buffer::FromString("foo"); return result; } @@ -1435,12 +1434,12 @@ class CancelTestServer : public FlightServerBase { public: Status ListFlights(const ServerCallContext&, const Criteria*, std::unique_ptr* listings) override { - *listings = arrow::internal::make_unique(); + *listings = std::make_unique(); return Status::OK(); } Status DoAction(const ServerCallContext&, const Action&, std::unique_ptr* result) override { - *result = arrow::internal::make_unique(); + *result = std::make_unique(); return Status::OK(); } Status ListActions(const ServerCallContext&, @@ -1450,7 +1449,7 @@ class CancelTestServer : public FlightServerBase { } Status DoGet(const ServerCallContext&, const Ticket&, std::unique_ptr* data_stream) override { - *data_stream = arrow::internal::make_unique(); + *data_stream = std::make_unique(); return Status::OK(); } }; diff --git a/cpp/src/arrow/flight/test_util.h b/cpp/src/arrow/flight/test_util.h index d5b774b4a37d3..c0c6d7514e169 100644 --- a/cpp/src/arrow/flight/test_util.h +++ b/cpp/src/arrow/flight/test_util.h @@ -30,7 +30,6 @@ #include "arrow/status.h" #include "arrow/testing/gtest_util.h" #include "arrow/testing/util.h" -#include "arrow/util/make_unique.h" #include "arrow/flight/client.h" #include "arrow/flight/client_auth.h" @@ -108,7 +107,7 @@ Status MakeServer(const Location& location, std::unique_ptr* s std::function make_server_options, std::function make_client_options, Args&&... server_args) { - *server = arrow::internal::make_unique(std::forward(server_args)...); + *server = std::make_unique(std::forward(server_args)...); FlightServerOptions server_options(location); RETURN_NOT_OK(make_server_options(&server_options)); RETURN_NOT_OK((*server)->Init(server_options)); diff --git a/cpp/src/arrow/flight/transport.cc b/cpp/src/arrow/flight/transport.cc index 7f0b1cf929d2c..a0281ffd61e60 100644 --- a/cpp/src/arrow/flight/transport.cc +++ b/cpp/src/arrow/flight/transport.cc @@ -17,6 +17,7 @@ #include "arrow/flight/transport.h" +#include #include #include @@ -26,7 +27,6 @@ #include "arrow/ipc/message.h" #include "arrow/result.h" #include "arrow/status.h" -#include "arrow/util/make_unique.h" namespace arrow { namespace flight { @@ -135,7 +135,7 @@ class TransportRegistry::Impl final { std::unordered_map server_factories_; }; -TransportRegistry::TransportRegistry() { impl_ = arrow::internal::make_unique(); } +TransportRegistry::TransportRegistry() { impl_ = std::make_unique(); } TransportRegistry::~TransportRegistry() = default; arrow::Result> TransportRegistry::MakeClient( const std::string& scheme) const { diff --git a/cpp/src/arrow/flight/transport/grpc/grpc_client.cc b/cpp/src/arrow/flight/transport/grpc/grpc_client.cc index 34c4ae91627cf..027cfb6022a16 100644 --- a/cpp/src/arrow/flight/transport/grpc/grpc_client.cc +++ b/cpp/src/arrow/flight/transport/grpc/grpc_client.cc @@ -43,7 +43,6 @@ #include "arrow/status.h" #include "arrow/util/base64.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/string.h" #include "arrow/util/uri.h" @@ -809,7 +808,7 @@ class GrpcClientImpl : public internal::ClientTransport { std::string str; RETURN_NOT_OK(internal::FromProto(pb_response, &str)); - return arrow::internal::make_unique(std::move(str)); + return std::make_unique(std::move(str)); } Status DoGet(const FlightCallOptions& options, const Ticket& ticket, diff --git a/cpp/src/arrow/flight/transport/ucx/ucx_client.cc b/cpp/src/arrow/flight/transport/ucx/ucx_client.cc index 80124123d4ae5..b5465d686abf0 100644 --- a/cpp/src/arrow/flight/transport/ucx/ucx_client.cc +++ b/cpp/src/arrow/flight/transport/ucx/ucx_client.cc @@ -33,6 +33,7 @@ #include #include +#include #include #include @@ -46,7 +47,6 @@ #include "arrow/result.h" #include "arrow/status.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/uri.h" namespace arrow { @@ -603,7 +603,7 @@ class UcxClientImpl : public arrow::flight::internal::ClientTransport { auto status = driver->StartCall(kMethodDoExchange); if (ARROW_PREDICT_TRUE(status.ok())) { *out = - arrow::internal::make_unique(this, std::move(connection)); + std::make_unique(this, std::move(connection)); return Status::OK(); } return MergeStatuses(std::move(status), ReturnConnection(std::move(connection))); @@ -621,7 +621,7 @@ class UcxClientImpl : public arrow::flight::internal::ClientTransport { reinterpret_cast(payload.data()), static_cast(payload.size()))); *stream = - arrow::internal::make_unique(this, std::move(connection)); + std::make_unique(this, std::move(connection)); return Status::OK(); }; @@ -637,7 +637,7 @@ class UcxClientImpl : public arrow::flight::internal::ClientTransport { auto status = driver->StartCall(kMethodDoPut); if (ARROW_PREDICT_TRUE(status.ok())) { - *out = arrow::internal::make_unique(this, std::move(connection)); + *out = std::make_unique(this, std::move(connection)); return Status::OK(); } return MergeStatuses(std::move(status), ReturnConnection(std::move(connection))); @@ -720,7 +720,7 @@ Status UcxClientStream::DoFinish() { } // namespace std::unique_ptr MakeUcxClientImpl() { - return arrow::internal::make_unique(); + return std::make_unique(); } } // namespace ucx diff --git a/cpp/src/arrow/flight/transport/ucx/ucx_internal.cc b/cpp/src/arrow/flight/transport/ucx/ucx_internal.cc index 318f6204ac9ee..f69002170ccdb 100644 --- a/cpp/src/arrow/flight/transport/ucx/ucx_internal.cc +++ b/cpp/src/arrow/flight/transport/ucx/ucx_internal.cc @@ -19,6 +19,7 @@ #include #include +#include #include #include @@ -28,7 +29,6 @@ #include "arrow/util/base64.h" #include "arrow/util/bit_util.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/uri.h" namespace arrow { @@ -621,7 +621,7 @@ class UcpCallDriver::Impl { // Preliminary profiling shows ~5% overhead just from mapping the buffer // alone (on Infiniband; it seems to be trivial for shared memory) request_param.datatype = ucp_dt_make_contig(1); - pending_send = arrow::internal::make_unique(); + pending_send = std::make_unique(); auto* pending_contig = reinterpret_cast(pending_send.get()); const int64_t body_length = std::max(payload.ipc_message.body_length, 1); @@ -654,7 +654,7 @@ class UcpCallDriver::Impl { } else { // IOV - let UCX use scatter-gather path request_param.datatype = UCP_DATATYPE_IOV; - pending_send = arrow::internal::make_unique(); + pending_send = std::make_unique(); auto* pending_iov = reinterpret_cast(pending_send.get()); pending_iov->payload = payload; @@ -911,7 +911,7 @@ class UcpCallDriver::Impl { // because we might run the callback synchronously (which might // free the buffer) when we call Push here. frame->buffer = - arrow::internal::make_unique(worker_, data, data_length); + std::make_unique(worker_, data, data_length); Push(std::move(frame)); return UCS_INPROGRESS; } diff --git a/cpp/src/arrow/flight/transport/ucx/ucx_server.cc b/cpp/src/arrow/flight/transport/ucx/ucx_server.cc index 398bc4381467c..6ee66e16e6486 100644 --- a/cpp/src/arrow/flight/transport/ucx/ucx_server.cc +++ b/cpp/src/arrow/flight/transport/ucx/ucx_server.cc @@ -18,6 +18,7 @@ #include "arrow/flight/transport/ucx/ucx_internal.h" #include +#include #include #include #include @@ -35,7 +36,6 @@ #include "arrow/status.h" #include "arrow/util/io_util.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/thread_pool.h" #include "arrow/util/uri.h" @@ -609,7 +609,7 @@ class UcxServerImpl : public arrow::flight::internal::ServerTransport { std::unique_ptr MakeUcxServerImpl( FlightServerBase* base, std::shared_ptr memory_manager) { - return arrow::internal::make_unique(base, memory_manager); + return std::make_unique(base, memory_manager); } #undef SERVER_RETURN_NOT_OK diff --git a/cpp/src/arrow/flight/transport/ucx/util_internal.cc b/cpp/src/arrow/flight/transport/ucx/util_internal.cc index ca4df21a055a4..5a7cb678b8ebb 100644 --- a/cpp/src/arrow/flight/transport/ucx/util_internal.cc +++ b/cpp/src/arrow/flight/transport/ucx/util_internal.cc @@ -22,6 +22,7 @@ #include #include +#include #include #include @@ -31,7 +32,6 @@ #include "arrow/util/bit_util.h" #include "arrow/util/io_util.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/uri.h" namespace arrow { diff --git a/cpp/src/arrow/flight/types.cc b/cpp/src/arrow/flight/types.cc index 2122e57ccc141..a06a3f4e1ba6a 100644 --- a/cpp/src/arrow/flight/types.cc +++ b/cpp/src/arrow/flight/types.cc @@ -29,7 +29,6 @@ #include "arrow/ipc/reader.h" #include "arrow/status.h" #include "arrow/table.h" -#include "arrow/util/make_unique.h" #include "arrow/util/uri.h" namespace arrow { @@ -154,7 +153,7 @@ arrow::Result> SchemaResult::GetSchema( arrow::Result> SchemaResult::Make(const Schema& schema) { std::string schema_in; RETURN_NOT_OK(internal::SchemaToString(schema, &schema_in)); - return arrow::internal::make_unique(std::move(schema_in)); + return std::make_unique(std::move(schema_in)); } Status SchemaResult::GetSchema(ipc::DictionaryMemo* dictionary_memo, diff --git a/cpp/src/arrow/gpu/cuda_context.cc b/cpp/src/arrow/gpu/cuda_context.cc index 52df8dbd8c76d..1376a4d6ceba0 100644 --- a/cpp/src/arrow/gpu/cuda_context.cc +++ b/cpp/src/arrow/gpu/cuda_context.cc @@ -31,7 +31,6 @@ #include "arrow/gpu/cuda_internal.h" #include "arrow/gpu/cuda_memory.h" #include "arrow/util/checked_cast.h" -#include "arrow/util/make_unique.h" namespace arrow { @@ -542,7 +541,7 @@ CudaContext::~CudaContext() {} Result> CudaContext::Allocate(int64_t nbytes) { uint8_t* data = nullptr; RETURN_NOT_OK(impl_->Allocate(nbytes, &data)); - return arrow::internal::make_unique(data, nbytes, this->shared_from_this(), + return std::make_unique(data, nbytes, this->shared_from_this(), true); } diff --git a/cpp/src/arrow/ipc/feather.cc b/cpp/src/arrow/ipc/feather.cc index ad4baf2527b9b..456b9d7b31a56 100644 --- a/cpp/src/arrow/ipc/feather.cc +++ b/cpp/src/arrow/ipc/feather.cc @@ -46,7 +46,6 @@ #include "arrow/util/bit_util.h" #include "arrow/util/checked_cast.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/visit_type_inline.h" #include "generated/feather_generated.h" @@ -54,7 +53,7 @@ namespace arrow { using internal::checked_cast; -using internal::make_unique; +using std::make_unique; class ExtensionType; diff --git a/cpp/src/arrow/ipc/reader.cc b/cpp/src/arrow/ipc/reader.cc index 0972d7e85ca7a..a1b17afaaf9f4 100644 --- a/cpp/src/arrow/ipc/reader.cc +++ b/cpp/src/arrow/ipc/reader.cc @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -52,7 +53,6 @@ #include "arrow/util/endian.h" #include "arrow/util/key_value_metadata.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/parallel.h" #include "arrow/util/string.h" #include "arrow/util/thread_pool.h" diff --git a/cpp/src/arrow/ipc/writer.cc b/cpp/src/arrow/ipc/writer.cc index d015ee3f4d98c..f97a2621954c2 100644 --- a/cpp/src/arrow/ipc/writer.cc +++ b/cpp/src/arrow/ipc/writer.cc @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -52,7 +53,6 @@ #include "arrow/util/endian.h" #include "arrow/util/key_value_metadata.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/parallel.h" #include "arrow/visit_array_inline.h" #include "arrow/visit_type_inline.h" @@ -1328,7 +1328,7 @@ Result> MakeStreamWriter( io::OutputStream* sink, const std::shared_ptr& schema, const IpcWriteOptions& options) { return std::make_shared( - ::arrow::internal::make_unique(sink, options), + std::make_unique(sink, options), schema, options, /*is_file_format=*/false); } @@ -1336,7 +1336,7 @@ Result> MakeStreamWriter( std::shared_ptr sink, const std::shared_ptr& schema, const IpcWriteOptions& options) { return std::make_shared( - ::arrow::internal::make_unique(std::move(sink), + std::make_unique(std::move(sink), options), schema, options, /*is_file_format=*/false); } @@ -1352,7 +1352,7 @@ Result> MakeFileWriter( const IpcWriteOptions& options, const std::shared_ptr& metadata) { return std::make_shared( - ::arrow::internal::make_unique(options, schema, + std::make_unique(options, schema, metadata, sink), schema, options, /*is_file_format=*/true); } @@ -1362,7 +1362,7 @@ Result> MakeFileWriter( const IpcWriteOptions& options, const std::shared_ptr& metadata) { return std::make_shared( - ::arrow::internal::make_unique( + std::make_unique( options, schema, metadata, std::move(sink)), schema, options, /*is_file_format=*/true); } @@ -1379,7 +1379,7 @@ namespace internal { Result> OpenRecordBatchWriter( std::unique_ptr sink, const std::shared_ptr& schema, const IpcWriteOptions& options) { - auto writer = ::arrow::internal::make_unique( + auto writer = std::make_unique( std::move(sink), schema, options, /*is_file_format=*/false); RETURN_NOT_OK(writer->Start()); return std::move(writer); @@ -1387,14 +1387,14 @@ Result> OpenRecordBatchWriter( Result> MakePayloadStreamWriter( io::OutputStream* sink, const IpcWriteOptions& options) { - return ::arrow::internal::make_unique(sink, options); + return std::make_unique(sink, options); } Result> MakePayloadFileWriter( io::OutputStream* sink, const std::shared_ptr& schema, const IpcWriteOptions& options, const std::shared_ptr& metadata) { - return ::arrow::internal::make_unique(options, schema, + return std::make_unique(options, schema, metadata, sink); } @@ -1446,7 +1446,7 @@ Result> SerializeSchema(const Schema& schema, MemoryPool auto options = IpcWriteOptions::Defaults(); const bool is_file_format = false; // indifferent as we don't write dictionaries internal::IpcFormatWriter writer( - ::arrow::internal::make_unique(stream.get()), schema, + std::make_unique(stream.get()), schema, options, is_file_format); RETURN_NOT_OK(writer.Start()); return stream->Finish(); diff --git a/cpp/src/arrow/json/chunker.cc b/cpp/src/arrow/json/chunker.cc index 362d8e13f5fec..b7721a2421162 100644 --- a/cpp/src/arrow/json/chunker.cc +++ b/cpp/src/arrow/json/chunker.cc @@ -28,11 +28,9 @@ #include "arrow/buffer.h" #include "arrow/json/options.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" namespace arrow { -using internal::make_unique; using std::string_view; namespace json { diff --git a/cpp/src/arrow/json/parser.cc b/cpp/src/arrow/json/parser.cc index 3774b578a83dc..903f7c1cc0d39 100644 --- a/cpp/src/arrow/json/parser.cc +++ b/cpp/src/arrow/json/parser.cc @@ -19,6 +19,7 @@ #include #include +#include #include #include #include @@ -36,7 +37,6 @@ #include "arrow/util/bitset_stack.h" #include "arrow/util/checked_cast.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/trie.h" #include "arrow/visit_type_inline.h" @@ -44,7 +44,7 @@ namespace arrow { using internal::BitsetStack; using internal::checked_cast; -using internal::make_unique; +using std::make_unique; using std::string_view; namespace json { diff --git a/cpp/src/arrow/type.cc b/cpp/src/arrow/type.cc index 25e323731964a..c4411236bc750 100644 --- a/cpp/src/arrow/type.cc +++ b/cpp/src/arrow/type.cc @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include // IWYU pragma: keep @@ -40,7 +41,6 @@ #include "arrow/util/hashing.h" #include "arrow/util/key_value_metadata.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/range.h" #include "arrow/util/vector.h" #include "arrow/visit_type_inline.h" @@ -1717,13 +1717,13 @@ class SchemaBuilder::Impl { SchemaBuilder::SchemaBuilder(ConflictPolicy policy, Field::MergeOptions field_merge_options) { - impl_ = internal::make_unique(policy, field_merge_options); + impl_ = std::make_unique(policy, field_merge_options); } SchemaBuilder::SchemaBuilder(std::vector> fields, ConflictPolicy policy, Field::MergeOptions field_merge_options) { - impl_ = internal::make_unique(std::move(fields), nullptr, policy, + impl_ = std::make_unique(std::move(fields), nullptr, policy, field_merge_options); } @@ -1734,7 +1734,7 @@ SchemaBuilder::SchemaBuilder(const std::shared_ptr& schema, ConflictPoli metadata = schema->metadata()->Copy(); } - impl_ = internal::make_unique(schema->fields(), std::move(metadata), policy, + impl_ = std::make_unique(schema->fields(), std::move(metadata), policy, field_merge_options); } diff --git a/cpp/src/arrow/util/async_util.cc b/cpp/src/arrow/util/async_util.cc index 3ac4519a6f0b3..f7f39704cb669 100644 --- a/cpp/src/arrow/util/async_util.cc +++ b/cpp/src/arrow/util/async_util.cc @@ -19,11 +19,11 @@ #include "arrow/util/future.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include #include #include +#include #include namespace arrow { @@ -69,7 +69,7 @@ class ThrottleImpl : public AsyncTaskScheduler::Throttle { std::unique_ptr AsyncTaskScheduler::MakeThrottle( int max_concurrent_cost) { - return ::arrow::internal::make_unique(max_concurrent_cost); + return std::make_unique(max_concurrent_cost); } namespace { @@ -107,13 +107,13 @@ class AsyncTaskSchedulerImpl : public AsyncTaskScheduler { throttle_(throttle), finish_callback_(std::move(finish_callback)) { if (parent == nullptr) { - owned_global_abort_ = ::arrow::internal::make_unique>(0); + owned_global_abort_ = std::make_unique>(0); global_abort_ = owned_global_abort_.get(); } else { global_abort_ = parent->global_abort_; } if (throttle != nullptr && !queue_) { - queue_ = ::arrow::internal::make_unique(); + queue_ = std::make_unique(); } } @@ -172,7 +172,7 @@ class AsyncTaskSchedulerImpl : public AsyncTaskScheduler { Throttle* throttle, std::unique_ptr queue) override { std::unique_ptr owned_child = - ::arrow::internal::make_unique( + std::make_unique( this, std::move(queue), throttle, std::move(finish_callback)); AsyncTaskScheduler* child = owned_child.get(); std::list>::iterator child_itr; @@ -351,7 +351,7 @@ class AsyncTaskSchedulerImpl : public AsyncTaskScheduler { std::unique_ptr AsyncTaskScheduler::Make( Throttle* throttle, std::unique_ptr queue) { - return ::arrow::internal::make_unique( + return std::make_unique( nullptr, std::move(queue), throttle, FnOnce()); } diff --git a/cpp/src/arrow/util/async_util.h b/cpp/src/arrow/util/async_util.h index 707f70d471fab..a00f263b92fed 100644 --- a/cpp/src/arrow/util/async_util.h +++ b/cpp/src/arrow/util/async_util.h @@ -21,9 +21,10 @@ #include "arrow/status.h" #include "arrow/util/functional.h" #include "arrow/util/future.h" -#include "arrow/util/make_unique.h" #include "arrow/util/mutex.h" +#include + namespace arrow { using internal::FnOnce; @@ -187,7 +188,7 @@ class ARROW_EXPORT AsyncTaskScheduler { template bool AddSimpleTask(Callable callable) { return AddTask( - ::arrow::internal::make_unique>(std::move(callable))); + std::make_unique>(std::move(callable))); } /// Signal that tasks are done being added /// diff --git a/cpp/src/arrow/util/async_util_test.cc b/cpp/src/arrow/util/async_util_test.cc index dfb688f70d175..714d5a3b11151 100644 --- a/cpp/src/arrow/util/async_util_test.cc +++ b/cpp/src/arrow/util/async_util_test.cc @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -31,7 +32,6 @@ #include "arrow/testing/future_util.h" #include "arrow/testing/gtest_util.h" #include "arrow/util/future.h" -#include "arrow/util/make_unique.h" namespace arrow { namespace util { @@ -234,7 +234,7 @@ TEST(AsyncTaskScheduler, EndWaitsForAddedButNotSubmittedTasks) { ASSERT_TRUE(was_run); /// Same test but block task by custom throttle - auto custom_throttle = ::arrow::internal::make_unique(); + auto custom_throttle = std::make_unique(); task_group = AsyncTaskScheduler::Make(custom_throttle.get()); was_run = false; ASSERT_TRUE(task_group->AddSimpleTask([&was_run] { @@ -438,7 +438,7 @@ TEST(AsyncTaskScheduler, Priority) { std::unique_ptr throttle = AsyncTaskScheduler::MakeThrottle(kNumConcurrentTasks); std::unique_ptr task_group = AsyncTaskScheduler::Make( - throttle.get(), ::arrow::internal::make_unique()); + throttle.get(), std::make_unique()); std::shared_ptr gate = GatingTask::Make(); int submit_order[kNumTasks]; @@ -450,7 +450,7 @@ TEST(AsyncTaskScheduler, Priority) { submit_order[order_index++] = priority; return gate->AsyncTask(); }; - auto task = ::arrow::internal::make_unique(task_exec, priority); + auto task = std::make_unique(task_exec, priority); task_group->AddTask(std::move(task)); } task_group->End(); diff --git a/cpp/src/arrow/util/converter.h b/cpp/src/arrow/util/converter.h index f87fbc7122f1e..8276a1d44d048 100644 --- a/cpp/src/arrow/util/converter.h +++ b/cpp/src/arrow/util/converter.h @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +#include #include #include #include @@ -25,7 +26,6 @@ #include "arrow/type.h" #include "arrow/type_traits.h" #include "arrow/util/checked_cast.h" -#include "arrow/util/make_unique.h" #include "arrow/visit_type_inline.h" namespace arrow { @@ -223,7 +223,7 @@ struct MakeConverterImpl { switch (t.value_type()->id()) { #define DICTIONARY_CASE(TYPE) \ case TYPE::type_id: \ - out = internal::make_unique< \ + out = std::make_unique< \ typename ConverterTrait::template dictionary_type>(); \ break; DICTIONARY_CASE(BooleanType); @@ -404,7 +404,7 @@ class Chunker { template static Result>> MakeChunker(std::unique_ptr converter) { - return internal::make_unique>(std::move(converter)); + return std::make_unique>(std::move(converter)); } } // namespace internal diff --git a/cpp/src/arrow/util/future_test.cc b/cpp/src/arrow/util/future_test.cc index 41b470c643190..4f0a23fe8fc58 100644 --- a/cpp/src/arrow/util/future_test.cc +++ b/cpp/src/arrow/util/future_test.cc @@ -38,7 +38,6 @@ #include "arrow/testing/gtest_util.h" #include "arrow/testing/matchers.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/thread_pool.h" namespace arrow { @@ -531,7 +530,7 @@ TEST(FutureStressTest, DeleteAfterWait) { for (int i = 0; i < kNumTasks; i++) { { std::unique_ptr> future = - internal::make_unique>(Future<>::Make()); + std::make_unique>(Future<>::Make()); std::thread t([&]() { SleepABit(); future->MarkFinished(); diff --git a/cpp/src/arrow/util/make_unique.h b/cpp/src/arrow/util/make_unique.h deleted file mode 100644 index 850e20409b9c1..0000000000000 --- a/cpp/src/arrow/util/make_unique.h +++ /dev/null @@ -1,42 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include -#include -#include - -namespace arrow { -namespace internal { - -template -typename std::enable_if::value, std::unique_ptr>::type make_unique( - A&&... args) { - return std::unique_ptr(new T(std::forward(args)...)); -} - -template -typename std::enable_if::value && std::extent::value == 0, - std::unique_ptr>::type -make_unique(std::size_t n) { - using value_type = typename std::remove_extent::type; - return std::unique_ptr(new value_type[n]); -} - -} // namespace internal -} // namespace arrow diff --git a/cpp/src/arrow/util/string_builder.cc b/cpp/src/arrow/util/string_builder.cc index 625ae00753476..ae526494141c4 100644 --- a/cpp/src/arrow/util/string_builder.cc +++ b/cpp/src/arrow/util/string_builder.cc @@ -17,19 +17,16 @@ #include "arrow/util/string_builder.h" +#include #include -#include "arrow/util/make_unique.h" - namespace arrow { -using internal::make_unique; - namespace util { namespace detail { StringStreamWrapper::StringStreamWrapper() - : sstream_(make_unique()), ostream_(*sstream_) {} + : sstream_(std::make_unique()), ostream_(*sstream_) {} StringStreamWrapper::~StringStreamWrapper() {} diff --git a/cpp/src/arrow/util/tracing.cc b/cpp/src/arrow/util/tracing.cc index 8bf21f688c4bd..8913339c0388a 100644 --- a/cpp/src/arrow/util/tracing.cc +++ b/cpp/src/arrow/util/tracing.cc @@ -18,12 +18,13 @@ #include "arrow/util/tracing.h" #include "arrow/util/config.h" -#include "arrow/util/make_unique.h" #include "arrow/util/tracing_internal.h" +#include + namespace arrow { -using internal::make_unique; +using std::make_unique; namespace util { namespace tracing { diff --git a/cpp/src/arrow/util/tracing_internal.cc b/cpp/src/arrow/util/tracing_internal.cc index 668a2aaba8bf9..58668cab18bc6 100644 --- a/cpp/src/arrow/util/tracing_internal.cc +++ b/cpp/src/arrow/util/tracing_internal.cc @@ -21,6 +21,7 @@ #include "arrow/util/tracing.h" #include +#include #include #include @@ -48,7 +49,6 @@ #include "arrow/util/io_util.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" namespace arrow { namespace internal { @@ -119,15 +119,15 @@ std::unique_ptr InitializeExporter() { if (maybe_env_var.ok()) { auto env_var = maybe_env_var.ValueOrDie(); if (env_var == "ostream") { - return arrow::internal::make_unique(); + return std::make_unique(); } else if (env_var == "otlp_http") { namespace otlp = opentelemetry::exporter::otlp; otlp::OtlpHttpExporterOptions opts; - return arrow::internal::make_unique(opts); + return std::make_unique(opts); } else if (env_var == "arrow_otlp_stdout") { - return arrow::internal::make_unique(&std::cout); + return std::make_unique(&std::cout); } else if (env_var == "arrow_otlp_stderr") { - return arrow::internal::make_unique(&std::cerr); + return std::make_unique(&std::cerr); } else if (!env_var.empty()) { ARROW_LOG(WARNING) << "Requested unknown backend " << kTracingBackendEnvVar << "=" << env_var; @@ -160,7 +160,7 @@ nostd::shared_ptr InitializeSdkTracerProvider() { options.schedule_delay_millis = std::chrono::milliseconds(500); options.max_export_batch_size = 16384; auto processor = - arrow::internal::make_unique(std::move(exporter), options); + std::make_unique(std::move(exporter), options); return std::make_shared(std::move(processor)); } return nostd::shared_ptr(); diff --git a/cpp/src/arrow/util/tracing_internal.h b/cpp/src/arrow/util/tracing_internal.h index d1da05671a8ed..7b97ebf7adb63 100644 --- a/cpp/src/arrow/util/tracing_internal.h +++ b/cpp/src/arrow/util/tracing_internal.h @@ -36,7 +36,6 @@ #include "arrow/util/async_generator.h" #include "arrow/util/iterator.h" -#include "arrow/util/make_unique.h" #include "arrow/util/tracing.h" #include "arrow/util/visibility.h" diff --git a/cpp/src/gandiva/engine.cc b/cpp/src/gandiva/engine.cc index 3bd52917756ad..7c81de1fc8c99 100644 --- a/cpp/src/gandiva/engine.cc +++ b/cpp/src/gandiva/engine.cc @@ -75,7 +75,6 @@ #pragma warning(pop) #endif -#include "arrow/util/make_unique.h" #include "gandiva/configuration.h" #include "gandiva/decimal_ir.h" #include "gandiva/exported_funcs_registry.h" @@ -121,7 +120,7 @@ Engine::Engine(const std::shared_ptr& conf, bool cached) : context_(std::move(ctx)), execution_engine_(std::move(engine)), - ir_builder_(arrow::internal::make_unique>(*context_)), + ir_builder_(std::make_unique>(*context_)), module_(module), types_(*context_), optimize_(conf->optimize()), @@ -148,8 +147,8 @@ Status Engine::Make(const std::shared_ptr& conf, bool cached, std::unique_ptr* out) { std::call_once(llvm_init_once_flag, InitOnce); - auto ctx = arrow::internal::make_unique(); - auto module = arrow::internal::make_unique("codegen", *ctx); + auto ctx = std::make_unique(); + auto module = std::make_unique("codegen", *ctx); // Capture before moving, ExecutionEngine does not allow retrieving the // original Module. diff --git a/cpp/src/parquet/arrow/path_internal.cc b/cpp/src/parquet/arrow/path_internal.cc index 0ef9eea1dabb7..06664ecf952bd 100644 --- a/cpp/src/parquet/arrow/path_internal.cc +++ b/cpp/src/parquet/arrow/path_internal.cc @@ -104,7 +104,6 @@ #include "arrow/util/bitmap_visit.h" #include "arrow/util/logging.h" #include "arrow/util/macros.h" -#include "arrow/util/make_unique.h" #include "arrow/visit_array_inline.h" #include "parquet/properties.h" @@ -878,9 +877,9 @@ class MultipathLevelBuilderImpl : public MultipathLevelBuilder { // static ::arrow::Result> MultipathLevelBuilder::Make( const ::arrow::Array& array, bool array_field_nullable) { - auto constructor = ::arrow::internal::make_unique(array_field_nullable); + auto constructor = std::make_unique(array_field_nullable); RETURN_NOT_OK(VisitArrayInline(array, constructor.get())); - return ::arrow::internal::make_unique( + return std::make_unique( array.data(), std::move(constructor)); } diff --git a/cpp/src/parquet/arrow/reader.cc b/cpp/src/parquet/arrow/reader.cc index 03470d4e8f1bb..31480133d7acd 100644 --- a/cpp/src/parquet/arrow/reader.cc +++ b/cpp/src/parquet/arrow/reader.cc @@ -19,6 +19,7 @@ #include #include +#include #include #include #include @@ -35,7 +36,6 @@ #include "arrow/util/future.h" #include "arrow/util/iterator.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "arrow/util/parallel.h" #include "arrow/util/range.h" #include "arrow/util/tracing_internal.h" @@ -994,7 +994,7 @@ Status FileReaderImpl::GetRecordBatchReader(const std::vector& row_groups, } } - *out = ::arrow::internal::make_unique( + *out = std::make_unique( ::arrow::MakeVectorIterator(std::move(batches)), std::move(batch_schema)); return Status::OK(); @@ -1038,7 +1038,7 @@ Status FileReaderImpl::GetRecordBatchReader(const std::vector& row_groups, [table, table_reader] { return table_reader->Next(); }); }); - *out = ::arrow::internal::make_unique( + *out = std::make_unique( ::arrow::MakeFlattenIterator(std::move(batches)), std::move(batch_schema)); return Status::OK(); diff --git a/cpp/src/parquet/arrow/writer.cc b/cpp/src/parquet/arrow/writer.cc index cf174dc61c859..035eca877a598 100644 --- a/cpp/src/parquet/arrow/writer.cc +++ b/cpp/src/parquet/arrow/writer.cc @@ -19,6 +19,7 @@ #include #include +#include #include #include #include @@ -33,7 +34,6 @@ #include "arrow/util/checked_cast.h" #include "arrow/util/key_value_metadata.h" #include "arrow/util/logging.h" -#include "arrow/util/make_unique.h" #include "parquet/arrow/path_internal.h" #include "parquet/arrow/reader_internal.h" @@ -166,7 +166,7 @@ class ArrowColumnWriterV2 { int chunk_index = 0; int64_t chunk_offset = 0; if (data.length() == 0) { - return ::arrow::internal::make_unique( + return std::make_unique( std::vector>{}, CalculateLeafCount(data.type().get()), row_group_writer); } @@ -239,7 +239,7 @@ class ArrowColumnWriterV2 { } values_written += chunk_write_size; } - return ::arrow::internal::make_unique( + return std::make_unique( std::move(builders), leaf_count, row_group_writer); } diff --git a/cpp/src/parquet/column_reader_test.cc b/cpp/src/parquet/column_reader_test.cc index 9fc034f41fd80..e7162eb981c81 100644 --- a/cpp/src/parquet/column_reader_test.cc +++ b/cpp/src/parquet/column_reader_test.cc @@ -27,7 +27,6 @@ #include #include "arrow/util/macros.h" -#include "arrow/util/make_unique.h" #include "parquet/column_page.h" #include "parquet/column_reader.h" #include "parquet/schema.h" @@ -511,7 +510,7 @@ TEST_F(TestPrimitiveReader, TestDictionaryEncodedPagesWithExposeEncoding) { int64_t total_indices = 0; int64_t indices_read = 0; int64_t value_size = values.size(); - auto indices = ::arrow::internal::make_unique(value_size); + auto indices = std::make_unique(value_size); while (total_indices < value_size && reader->HasNext()) { const ByteArray* tmp_dict = nullptr; int32_t tmp_dict_len = 0; @@ -564,7 +563,7 @@ TEST_F(TestPrimitiveReader, TestNonDictionaryEncodedPagesWithExposeEncoding) { const ByteArray* dict = nullptr; int32_t dict_len = 0; int64_t indices_read = 0; - auto indices = ::arrow::internal::make_unique(value_size); + auto indices = std::make_unique(value_size); // Dictionary cannot be exposed when it's not fully dictionary encoded EXPECT_THROW(reader->ReadBatchWithDictionary(value_size, /*def_levels=*/nullptr, /*rep_levels=*/nullptr, indices.get(), diff --git a/cpp/src/parquet/reader_test.cc b/cpp/src/parquet/reader_test.cc index a43238d1369c6..8f7ddb7df9708 100644 --- a/cpp/src/parquet/reader_test.cc +++ b/cpp/src/parquet/reader_test.cc @@ -31,7 +31,6 @@ #include "arrow/testing/gtest_util.h" #include "arrow/testing/random.h" #include "arrow/util/checked_cast.h" -#include "arrow/util/make_unique.h" #include "parquet/column_reader.h" #include "parquet/column_scanner.h" @@ -612,7 +611,7 @@ TEST(TestFileReader, BufferedReadsWithDictionary) { row_group->ColumnWithExposeEncoding(0, ExposedEncoding::DICTIONARY)); EXPECT_EQ(col_reader->GetExposedEncoding(), ExposedEncoding::DICTIONARY); - auto indices = ::arrow::internal::make_unique(num_rows); + auto indices = std::make_unique(num_rows); const double* dict = nullptr; int32_t dict_len = 0; for (int row_index = 0; row_index < num_rows; ++row_index) { diff --git a/cpp/src/skyhook/protocol/rados_protocol.h b/cpp/src/skyhook/protocol/rados_protocol.h index 3e5fac7640b65..6a0b2ea0067ff 100644 --- a/cpp/src/skyhook/protocol/rados_protocol.h +++ b/cpp/src/skyhook/protocol/rados_protocol.h @@ -19,16 +19,17 @@ #include #include "arrow/status.h" -#include "arrow/util/make_unique.h" #include "skyhook/client/file_skyhook.h" +#include + namespace skyhook { namespace rados { class IoCtxInterface { public: - IoCtxInterface() { ioCtx = arrow::internal::make_unique(); } + IoCtxInterface() { ioCtx = std::make_unique(); } /// \brief Read from a RADOS object. /// /// \param[in] oid the ID of the object to read. @@ -60,7 +61,7 @@ class IoCtxInterface { class RadosInterface { public: - RadosInterface() { cluster = arrow::internal::make_unique(); } + RadosInterface() { cluster = std::make_unique(); } /// Initializes a cluster handle. arrow::Status init2(const char* const name, const char* const clustername, uint64_t flags); @@ -83,8 +84,8 @@ class RadosConn { public: explicit RadosConn(std::shared_ptr ctx) : ctx(std::move(ctx)), - rados(arrow::internal::make_unique()), - io_ctx(arrow::internal::make_unique()), + rados(std::make_unique()), + io_ctx(std::make_unique()), connected(false) {} ~RadosConn(); /// Connect to the Ceph cluster.