From 3a4fcff9a681eca258ef0eb2cc2ad23ddffcc207 Mon Sep 17 00:00:00 2001 From: William Ayd Date: Fri, 24 May 2024 12:07:38 -0400 Subject: [PATCH] GH-41478: [C++] Clean up more redundant move warnings (#41487) ### Rationale for this change Minor warning cleanup for downstream libraries trying to get warning-free builds ### What changes are included in this PR? Removed redundant std::move from return statements ### Are these changes tested? Builds cleanly ### Are there any user-facing changes? No * GitHub Issue: #41478 Authored-by: Will Ayd Signed-off-by: Benjamin Kietzman --- cpp/cmake_modules/SetupCxxFlags.cmake | 3 ++- cpp/src/arrow/acero/aggregate_internal.cc | 6 ++--- cpp/src/arrow/acero/backpressure_handler.h | 2 +- cpp/src/arrow/acero/hash_aggregate_test.cc | 2 +- cpp/src/arrow/acero/hash_join.cc | 2 +- cpp/src/arrow/acero/hash_join_node.cc | 4 +-- cpp/src/arrow/acero/order_by_impl.cc | 4 +-- cpp/src/arrow/acero/swiss_join.cc | 2 +- cpp/src/arrow/acero/task_util.cc | 2 +- cpp/src/arrow/acero/tpch_node.cc | 14 +++++----- cpp/src/arrow/adapters/orc/adapter.cc | 4 +-- cpp/src/arrow/adapters/orc/util.cc | 6 ++--- cpp/src/arrow/array/array_nested.cc | 2 +- cpp/src/arrow/array/builder_base.h | 6 ++--- cpp/src/arrow/array/builder_run_end.cc | 3 +-- cpp/src/arrow/array/concatenate.cc | 3 ++- cpp/src/arrow/array/util.cc | 3 ++- cpp/src/arrow/buffer.cc | 12 ++++++--- cpp/src/arrow/c/bridge_test.cc | 2 +- cpp/src/arrow/compute/exec.cc | 8 +++--- cpp/src/arrow/compute/expression.cc | 4 +-- cpp/src/arrow/compute/function_internal.h | 5 ++-- cpp/src/arrow/compute/kernel.cc | 2 +- .../arrow/compute/kernels/hash_aggregate.cc | 18 +++++++------ .../compute/kernels/scalar_arithmetic.cc | 4 +-- .../compute/kernels/scalar_string_ascii.cc | 10 +++---- cpp/src/arrow/compute/kernels/vector_hash.cc | 3 ++- cpp/src/arrow/compute/row/grouper.cc | 8 +++--- cpp/src/arrow/csv/reader.cc | 4 +-- cpp/src/arrow/dataset/dataset.cc | 2 +- cpp/src/arrow/dataset/dataset_writer.cc | 3 +-- cpp/src/arrow/dataset/file_csv.cc | 2 +- cpp/src/arrow/dataset/file_parquet.cc | 9 ++++--- cpp/src/arrow/dataset/scan_node.cc | 6 ++--- cpp/src/arrow/device.cc | 4 +-- .../engine/substrait/expression_internal.cc | 26 +++++++++---------- .../substrait/extended_expression_internal.cc | 6 ++--- .../arrow/engine/substrait/extension_set.cc | 10 +++---- .../arrow/engine/substrait/plan_internal.cc | 2 +- .../engine/substrait/relation_internal.cc | 10 +++---- cpp/src/arrow/engine/substrait/serde.cc | 2 +- .../arrow/engine/substrait/type_internal.cc | 4 +-- cpp/src/arrow/field_ref_test.cc | 2 +- cpp/src/arrow/filesystem/azurefs.cc | 4 +-- cpp/src/arrow/filesystem/localfs.cc | 2 +- cpp/src/arrow/filesystem/localfs_test.cc | 2 +- cpp/src/arrow/filesystem/s3fs.cc | 5 ++-- cpp/src/arrow/filesystem/util_internal.cc | 2 +- cpp/src/arrow/flight/client.cc | 4 +-- cpp/src/arrow/flight/sql/client.cc | 2 +- cpp/src/arrow/gpu/cuda_memory.cc | 2 +- cpp/src/arrow/integration/json_internal.cc | 4 +-- cpp/src/arrow/io/buffered.cc | 3 ++- cpp/src/arrow/io/compressed.cc | 5 ++-- cpp/src/arrow/io/compressed_test.cc | 2 +- cpp/src/arrow/io/file.cc | 6 +++-- cpp/src/arrow/io/hdfs.cc | 6 +++-- cpp/src/arrow/io/stdio.cc | 3 ++- cpp/src/arrow/ipc/message.cc | 18 ++++++------- cpp/src/arrow/ipc/metadata_internal.cc | 6 ++--- cpp/src/arrow/ipc/metadata_internal.h | 3 ++- cpp/src/arrow/ipc/reader.cc | 11 ++++---- cpp/src/arrow/ipc/writer.cc | 3 ++- cpp/src/arrow/scalar.cc | 8 +++--- cpp/src/arrow/table_builder.cc | 2 +- cpp/src/arrow/testing/builder.h | 2 +- cpp/src/arrow/type.cc | 4 +-- cpp/src/arrow/util/align_util.cc | 17 ++++++------ cpp/src/arrow/util/async_generator.h | 2 +- cpp/src/arrow/util/bit_util_benchmark.cc | 2 +- cpp/src/arrow/util/bitmap_builders.cc | 6 +++-- cpp/src/arrow/util/bitmap_reader_benchmark.cc | 2 +- cpp/src/arrow/util/compression.cc | 2 +- cpp/src/arrow/util/decimal.cc | 4 +-- cpp/src/arrow/util/future.cc | 2 +- cpp/src/arrow/util/future.h | 2 +- cpp/src/arrow/util/io_util.cc | 6 ++--- cpp/src/arrow/util/iterator.h | 4 +-- cpp/src/arrow/util/vector.h | 6 ++--- cpp/src/gandiva/function_registry.cc | 2 +- cpp/src/gandiva/llvm_generator.cc | 2 +- .../parquet/arrow/arrow_reader_writer_test.cc | 2 +- cpp/src/parquet/encoding.cc | 2 +- .../parquet/encryption/file_key_unwrapper.cc | 2 +- cpp/src/parquet/platform.cc | 2 +- cpp/src/parquet/properties.cc | 2 +- cpp/src/skyhook/cls/cls_skyhook.cc | 2 +- 87 files changed, 218 insertions(+), 198 deletions(-) diff --git a/cpp/cmake_modules/SetupCxxFlags.cmake b/cpp/cmake_modules/SetupCxxFlags.cmake index ea357b47794ce..e2e1c4412abd0 100644 --- a/cpp/cmake_modules/SetupCxxFlags.cmake +++ b/cpp/cmake_modules/SetupCxxFlags.cmake @@ -330,8 +330,9 @@ if("${BUILD_WARNING_LEVEL}" STREQUAL "CHECKIN") set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wall") set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wno-conversion") set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wno-sign-conversion") - set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wunused-result") set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wdate-time") + string(APPEND CXX_ONLY_FLAGS " -Wredundant-move") + set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wunused-result") elseif(CMAKE_CXX_COMPILER_ID STREQUAL "Intel" OR CMAKE_CXX_COMPILER_ID STREQUAL "IntelLLVM") if(WIN32) diff --git a/cpp/src/arrow/acero/aggregate_internal.cc b/cpp/src/arrow/acero/aggregate_internal.cc index 9c4b7fe5ae98c..0c1bc3db365a6 100644 --- a/cpp/src/arrow/acero/aggregate_internal.cc +++ b/cpp/src/arrow/acero/aggregate_internal.cc @@ -102,7 +102,7 @@ Result> InitKernel(const HashAggregateKernel* kerne ARROW_ASSIGN_OR_RAISE( auto state, kernel->init(&kernel_ctx, KernelInitArgs{kernel, aggr_in_types, options})); - return std::move(state); + return state; } Result> GetKernels( @@ -129,7 +129,7 @@ Result>> InitKernels( ARROW_ASSIGN_OR_RAISE(states[i], InitKernel(kernels[i], ctx, aggregates[i], in_types[i])); } - return std::move(states); + return states; } Result ResolveKernels( @@ -242,7 +242,7 @@ Result> ExtractValues(const ExecBatch& input_batch, DCHECK(false); } } - return std::move(values); + return values; } } // namespace aggregate diff --git a/cpp/src/arrow/acero/backpressure_handler.h b/cpp/src/arrow/acero/backpressure_handler.h index 178272315d7fb..db6c3799354af 100644 --- a/cpp/src/arrow/acero/backpressure_handler.h +++ b/cpp/src/arrow/acero/backpressure_handler.h @@ -45,7 +45,7 @@ class BackpressureHandler { } BackpressureHandler backpressure_handler(input, low_threshold, high_threshold, std::move(backpressure_control)); - return std::move(backpressure_handler); + return backpressure_handler; } void Handle(size_t start_level, size_t end_level) { diff --git a/cpp/src/arrow/acero/hash_aggregate_test.cc b/cpp/src/arrow/acero/hash_aggregate_test.cc index d529f443319b9..743cb20d1960d 100644 --- a/cpp/src/arrow/acero/hash_aggregate_test.cc +++ b/cpp/src/arrow/acero/hash_aggregate_test.cc @@ -318,7 +318,7 @@ Result RunGroupBy(const BatchesWithSchema& input, { {"source", SourceNodeOptions{input.schema, input.gen(use_threads, /*slow=*/false)}}, - {"aggregate", AggregateNodeOptions{std::move(aggregates), std::move(keys), + {"aggregate", AggregateNodeOptions{aggregates, std::move(keys), std::move(segment_keys)}}, {"sink", SinkNodeOptions{&sink_gen}}, }) diff --git a/cpp/src/arrow/acero/hash_join.cc b/cpp/src/arrow/acero/hash_join.cc index 296b2c56e00f4..5aa70a23f7c9e 100644 --- a/cpp/src/arrow/acero/hash_join.cc +++ b/cpp/src/arrow/acero/hash_join.cc @@ -791,7 +791,7 @@ class HashJoinBasicImpl : public HashJoinImpl { Result> HashJoinImpl::MakeBasic() { std::unique_ptr impl{new HashJoinBasicImpl()}; - return std::move(impl); + return impl; } } // namespace acero diff --git a/cpp/src/arrow/acero/hash_join_node.cc b/cpp/src/arrow/acero/hash_join_node.cc index 06405f16c8d4c..67f902e64be93 100644 --- a/cpp/src/arrow/acero/hash_join_node.cc +++ b/cpp/src/arrow/acero/hash_join_node.cc @@ -351,7 +351,7 @@ Result HashJoinSchema::BindFilter(Expression filter, const Schema& right_schema, ExecContext* exec_context) { if (filter.IsBound() || filter == literal(true)) { - return std::move(filter); + return filter; } // Step 1: Construct filter schema FieldVector fields; @@ -386,7 +386,7 @@ Result HashJoinSchema::BindFilter(Expression filter, filter.ToString(), " evaluates to ", filter.type()->ToString()); } - return std::move(filter); + return filter; } Expression HashJoinSchema::RewriteFilterToUseFilterSchema( diff --git a/cpp/src/arrow/acero/order_by_impl.cc b/cpp/src/arrow/acero/order_by_impl.cc index 2c624f6ab895f..1165799fc6610 100644 --- a/cpp/src/arrow/acero/order_by_impl.cc +++ b/cpp/src/arrow/acero/order_by_impl.cc @@ -93,14 +93,14 @@ Result> OrderByImpl::MakeSort( ExecContext* ctx, const std::shared_ptr& output_schema, const SortOptions& options) { std::unique_ptr impl{new SortBasicImpl(ctx, output_schema, options)}; - return std::move(impl); + return impl; } Result> OrderByImpl::MakeSelectK( ExecContext* ctx, const std::shared_ptr& output_schema, const SelectKOptions& options) { std::unique_ptr impl{new SelectKBasicImpl(ctx, output_schema, options)}; - return std::move(impl); + return impl; } } // namespace acero diff --git a/cpp/src/arrow/acero/swiss_join.cc b/cpp/src/arrow/acero/swiss_join.cc index 17c5212697339..732deb72861d6 100644 --- a/cpp/src/arrow/acero/swiss_join.cc +++ b/cpp/src/arrow/acero/swiss_join.cc @@ -2985,7 +2985,7 @@ class SwissJoin : public HashJoinImpl { Result> HashJoinImpl::MakeSwiss() { std::unique_ptr impl{new SwissJoin()}; - return std::move(impl); + return impl; } } // namespace acero diff --git a/cpp/src/arrow/acero/task_util.cc b/cpp/src/arrow/acero/task_util.cc index 4d8e9ecf76597..85378eaeeb27c 100644 --- a/cpp/src/arrow/acero/task_util.cc +++ b/cpp/src/arrow/acero/task_util.cc @@ -424,7 +424,7 @@ void TaskSchedulerImpl::Abort(AbortContinuationImpl impl) { std::unique_ptr TaskScheduler::Make() { std::unique_ptr impl{new TaskSchedulerImpl()}; - return std::move(impl); + return impl; } } // namespace acero diff --git a/cpp/src/arrow/acero/tpch_node.cc b/cpp/src/arrow/acero/tpch_node.cc index 9797a082b49d2..137b62ad38a95 100644 --- a/cpp/src/arrow/acero/tpch_node.cc +++ b/cpp/src/arrow/acero/tpch_node.cc @@ -336,7 +336,7 @@ Result TpchPseudotext::GenerateComments(size_t num_comments, size_t min_l } ArrayData ad(utf8(), num_comments, {nullptr, std::move(offset_buffer), std::move(comment_buffer)}); - return std::move(ad); + return ad; } bool TpchPseudotext::GenerateWord(int64_t& offset, random::pcg32_fast& rng, char* arr, @@ -611,7 +611,7 @@ Result RandomVString(random::pcg32_fast& rng, int64_t num_rows, int32_t m for (int32_t i = 0; i < offsets[num_rows]; i++) str[i] = alpha_numerics[char_dist(rng)]; ArrayData ad(utf8(), num_rows, {nullptr, std::move(offset_buff), std::move(str_buff)}); - return std::move(ad); + return ad; } void GeneratePhoneNumber(char* out, random::pcg32_fast& rng, int32_t country) { @@ -677,7 +677,7 @@ class PartAndPartSupplierGenerator { if (!part_output_queue_.empty()) { ExecBatch batch = std::move(part_output_queue_.front()); part_output_queue_.pop(); - return std::move(batch); + return batch; } else if (part_rows_generated_ == part_rows_to_generate_) { return std::nullopt; } else { @@ -732,7 +732,7 @@ class PartAndPartSupplierGenerator { if (!partsupp_output_queue_.empty()) { ExecBatch result = std::move(partsupp_output_queue_.front()); partsupp_output_queue_.pop(); - return std::move(result); + return result; } } { @@ -1337,7 +1337,7 @@ class OrdersAndLineItemGenerator { if (!orders_output_queue_.empty()) { ExecBatch batch = std::move(orders_output_queue_.front()); orders_output_queue_.pop(); - return std::move(batch); + return batch; } else if (orders_rows_generated_ == orders_rows_to_generate_) { return std::nullopt; } else { @@ -1401,12 +1401,12 @@ class OrdersAndLineItemGenerator { if (from_queue) { ARROW_DCHECK(queued.length <= batch_size_); tld.first_batch_offset = queued.length; - if (queued.length == batch_size_) return std::move(queued); + if (queued.length == batch_size_) return queued; } { std::lock_guard lock(orders_output_queue_mutex_); if (orders_rows_generated_ == orders_rows_to_generate_) { - if (from_queue) return std::move(queued); + if (from_queue) return queued; return std::nullopt; } diff --git a/cpp/src/arrow/adapters/orc/adapter.cc b/cpp/src/arrow/adapters/orc/adapter.cc index 98784450b3cce..25759f8471365 100644 --- a/cpp/src/arrow/adapters/orc/adapter.cc +++ b/cpp/src/arrow/adapters/orc/adapter.cc @@ -566,7 +566,7 @@ Result> ORCFileReader::Open( #endif auto result = std::unique_ptr(new ORCFileReader()); RETURN_NOT_OK(result->impl_->Open(file, pool)); - return std::move(result); + return result; } Result> ORCFileReader::ReadMetadata() { @@ -837,7 +837,7 @@ Result> ORCFileWriter::Open( std::unique_ptr(new ORCFileWriter()); Status status = result->impl_->Open(output_stream, writer_options); RETURN_NOT_OK(status); - return std::move(result); + return result; } Status ORCFileWriter::Write(const Table& table) { return impl_->Write(table); } diff --git a/cpp/src/arrow/adapters/orc/util.cc b/cpp/src/arrow/adapters/orc/util.cc index 2a74bec1aa6fd..5bfe257ac7bad 100644 --- a/cpp/src/arrow/adapters/orc/util.cc +++ b/cpp/src/arrow/adapters/orc/util.cc @@ -1026,7 +1026,7 @@ Result> GetOrcType(const DataType& type) { SetAttributes(*it, orc_subtype.get()); out_type->addStructField(field_name, std::move(orc_subtype)); } - return std::move(out_type); + return out_type; } case Type::type::MAP: { const auto& key_field = checked_cast(type).key_field(); @@ -1048,7 +1048,7 @@ Result> GetOrcType(const DataType& type) { SetAttributes(arrow_field, orc_subtype.get()); out_type->addUnionChild(std::move(orc_subtype)); } - return std::move(out_type); + return out_type; } default: { return Status::NotImplemented("Unknown or unsupported Arrow type: ", @@ -1195,7 +1195,7 @@ Result> GetOrcType(const Schema& schema) { SetAttributes(field, orc_subtype.get()); out_type->addStructField(field->name(), std::move(orc_subtype)); } - return std::move(out_type); + return out_type; } Result> GetFieldMetadata( diff --git a/cpp/src/arrow/array/array_nested.cc b/cpp/src/arrow/array/array_nested.cc index 1be771d8228d9..67a499c2b8277 100644 --- a/cpp/src/arrow/array/array_nested.cc +++ b/cpp/src/arrow/array/array_nested.cc @@ -1177,7 +1177,7 @@ void SparseUnionArray::SetData(std::shared_ptr data) { } void DenseUnionArray::SetData(const std::shared_ptr& data) { - this->UnionArray::SetData(std::move(data)); + this->UnionArray::SetData(data); ARROW_CHECK_EQ(data_->type->id(), Type::DENSE_UNION); ARROW_CHECK_EQ(data_->buffers.size(), 3); diff --git a/cpp/src/arrow/array/builder_base.h b/cpp/src/arrow/array/builder_base.h index 3a85318735f80..ecd2136f5d20b 100644 --- a/cpp/src/arrow/array/builder_base.h +++ b/cpp/src/arrow/array/builder_base.h @@ -332,7 +332,7 @@ inline Result> MakeBuilder( const std::shared_ptr& type, MemoryPool* pool = default_memory_pool()) { std::unique_ptr out; ARROW_RETURN_NOT_OK(MakeBuilder(pool, type, &out)); - return std::move(out); + return out; } /// \brief Construct an empty ArrayBuilder corresponding to the data @@ -346,7 +346,7 @@ inline Result> MakeBuilderExactIndex( const std::shared_ptr& type, MemoryPool* pool = default_memory_pool()) { std::unique_ptr out; ARROW_RETURN_NOT_OK(MakeBuilderExactIndex(pool, type, &out)); - return std::move(out); + return out; } /// \brief Construct an empty DictionaryBuilder initialized optionally @@ -365,7 +365,7 @@ inline Result> MakeDictionaryBuilder( MemoryPool* pool = default_memory_pool()) { std::unique_ptr out; ARROW_RETURN_NOT_OK(MakeDictionaryBuilder(pool, type, dictionary, &out)); - return std::move(out); + return out; } } // namespace arrow diff --git a/cpp/src/arrow/array/builder_run_end.cc b/cpp/src/arrow/array/builder_run_end.cc index cff8d72952385..ed384123d8b87 100644 --- a/cpp/src/arrow/array/builder_run_end.cc +++ b/cpp/src/arrow/array/builder_run_end.cc @@ -162,8 +162,7 @@ Status RunCompressorBuilder::FinishInternal(std::shared_ptr* out) { RunEndEncodedBuilder::ValueRunBuilder::ValueRunBuilder( MemoryPool* pool, const std::shared_ptr& value_builder, const std::shared_ptr& value_type, RunEndEncodedBuilder& ree_builder) - : RunCompressorBuilder(pool, std::move(value_builder), std::move(value_type)), - ree_builder_(ree_builder) {} + : RunCompressorBuilder(pool, value_builder, value_type), ree_builder_(ree_builder) {} RunEndEncodedBuilder::RunEndEncodedBuilder( MemoryPool* pool, const std::shared_ptr& run_end_builder, diff --git a/cpp/src/arrow/array/concatenate.cc b/cpp/src/arrow/array/concatenate.cc index 44d58cc0bdebc..87e55246c78fe 100644 --- a/cpp/src/arrow/array/concatenate.cc +++ b/cpp/src/arrow/array/concatenate.cc @@ -522,7 +522,8 @@ class ConcatenateImpl { } out_data += data->length * index_width; } - return std::move(out); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(out)); } Status Visit(const DictionaryType& d) { diff --git a/cpp/src/arrow/array/util.cc b/cpp/src/arrow/array/util.cc index 41cd6a1c0b260..b56ea25f9e421 100644 --- a/cpp/src/arrow/array/util.cc +++ b/cpp/src/arrow/array/util.cc @@ -125,7 +125,8 @@ class ArrayDataEndianSwapper { for (int64_t i = 0; i < length; i++) { out_data[i] = bit_util::ByteSwap(in_data[i]); } - return std::move(out_buffer); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(out_buffer)); } template diff --git a/cpp/src/arrow/buffer.cc b/cpp/src/arrow/buffer.cc index 1bd789b7cafe6..0eb22a9d1553d 100644 --- a/cpp/src/arrow/buffer.cc +++ b/cpp/src/arrow/buffer.cc @@ -41,7 +41,8 @@ Result> Buffer::CopySlice(const int64_t start, ARROW_ASSIGN_OR_RAISE(auto new_buffer, AllocateResizableBuffer(nbytes, pool)); std::memcpy(new_buffer->mutable_data(), data() + start, static_cast(nbytes)); - return std::move(new_buffer); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(new_buffer)); } Buffer::Buffer() : Buffer(memory_pool::internal::kZeroSizeArea, 0) {} @@ -185,7 +186,8 @@ Result> AllocateBitmap(int64_t length, MemoryPool* pool) if (buf->size() > 0) { buf->mutable_data()[buf->size() - 1] = 0; } - return std::move(buf); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(buf)); } Result> AllocateEmptyBitmap(int64_t length, MemoryPool* pool) { @@ -197,7 +199,8 @@ Result> AllocateEmptyBitmap(int64_t length, int64_t alig ARROW_ASSIGN_OR_RAISE(auto buf, AllocateBuffer(bit_util::BytesForBits(length), alignment, pool)); memset(buf->mutable_data(), 0, static_cast(buf->size())); - return std::move(buf); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(buf)); } Status AllocateEmptyBitmap(int64_t length, std::shared_ptr* out) { @@ -219,7 +222,8 @@ Result> ConcatenateBuffers( out_data += buffer->size(); } } - return std::move(out); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(out)); } } // namespace arrow diff --git a/cpp/src/arrow/c/bridge_test.cc b/cpp/src/arrow/c/bridge_test.cc index e3ec262422ba6..09bb524adbdf0 100644 --- a/cpp/src/arrow/c/bridge_test.cc +++ b/cpp/src/arrow/c/bridge_test.cc @@ -1362,7 +1362,7 @@ class MyMemoryManager : public CPUMemoryManager { if (buf.size() > 0) { memcpy(dest->mutable_data(), buf.data(), static_cast(buf.size())); } - return std::move(dest); + return dest; } }; diff --git a/cpp/src/arrow/compute/exec.cc b/cpp/src/arrow/compute/exec.cc index f2e4578383122..05c4936482b0b 100644 --- a/cpp/src/arrow/compute/exec.cc +++ b/cpp/src/arrow/compute/exec.cc @@ -923,7 +923,7 @@ class ScalarExecutor : public KernelExecutorImpl { DCHECK(output.is_array_data()); // Emit a result for each chunk - RETURN_NOT_OK(EmitResult(std::move(output.array_data()), listener)); + RETURN_NOT_OK(EmitResult(output.array_data(), listener)); } return Status::OK(); } @@ -1107,7 +1107,7 @@ class VectorExecutor : public KernelExecutorImpl { RETURN_NOT_OK(PropagateNulls(kernel_ctx_, span, out.array_data().get())); } RETURN_NOT_OK(kernel_->exec(kernel_ctx_, span, &out)); - return EmitResult(std::move(out.array_data()), listener); + return EmitResult(out.array_data(), listener); } Status ExecChunked(const ExecBatch& batch, ExecListener* listener) { @@ -1116,10 +1116,10 @@ class VectorExecutor : public KernelExecutorImpl { ARROW_ASSIGN_OR_RAISE(out.value, PrepareOutput(batch.length)); RETURN_NOT_OK(kernel_->exec_chunked(kernel_ctx_, batch, &out)); if (out.is_array()) { - return EmitResult(std::move(out.array()), listener); + return EmitResult(out.array(), listener); } else { DCHECK(out.is_chunked_array()); - return EmitResult(std::move(out.chunked_array()), listener); + return EmitResult(out.chunked_array(), listener); } } diff --git a/cpp/src/arrow/compute/expression.cc b/cpp/src/arrow/compute/expression.cc index 532869b3453a7..b1d914ce873cc 100644 --- a/cpp/src/arrow/compute/expression.cc +++ b/cpp/src/arrow/compute/expression.cc @@ -1645,7 +1645,7 @@ Expression and_(const std::vector& operands) { Expression folded = operands.front(); for (auto it = operands.begin() + 1; it != operands.end(); ++it) { - folded = and_(std::move(folded), std::move(*it)); + folded = and_(std::move(folded), *it); } return folded; } @@ -1659,7 +1659,7 @@ Expression or_(const std::vector& operands) { Expression folded = operands.front(); for (auto it = operands.begin() + 1; it != operands.end(); ++it) { - folded = or_(std::move(folded), std::move(*it)); + folded = or_(std::move(folded), *it); } return folded; } diff --git a/cpp/src/arrow/compute/function_internal.h b/cpp/src/arrow/compute/function_internal.h index 653273ef0fac2..9d8928466baa5 100644 --- a/cpp/src/arrow/compute/function_internal.h +++ b/cpp/src/arrow/compute/function_internal.h @@ -684,12 +684,13 @@ const FunctionOptionsType* GetFunctionOptionsType(const Properties&... propertie auto options = std::make_unique(); RETURN_NOT_OK( FromStructScalarImpl(options.get(), scalar, properties_).status_); - return std::move(options); + // R build with openSUSE155 requires an explicit unique_ptr construction + return std::unique_ptr(std::move(options)); } std::unique_ptr Copy(const FunctionOptions& options) const override { auto out = std::make_unique(); CopyImpl(out.get(), checked_cast(options), properties_); - return std::move(out); + return out; } private: diff --git a/cpp/src/arrow/compute/kernel.cc b/cpp/src/arrow/compute/kernel.cc index fd554ba3d83c5..9cc5cc10917ee 100644 --- a/cpp/src/arrow/compute/kernel.cc +++ b/cpp/src/arrow/compute/kernel.cc @@ -75,7 +75,7 @@ Result> ScalarAggregateKernel::MergeAll( for (auto& state : states) { RETURN_NOT_OK(kernel->merge(ctx, std::move(*state), out.get())); } - return std::move(out); + return out; } // ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 5052d8dd66694..54cd695421a93 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -83,7 +83,8 @@ Result> HashAggregateInit(KernelContext* ctx, const KernelInitArgs& args) { auto impl = std::make_unique(); RETURN_NOT_OK(impl->Init(ctx->exec_context(), args)); - return std::move(impl); + // R build with openSUSE155 requires an explicit unique_ptr construction + return std::unique_ptr(std::move(impl)); } Status HashAggregateResize(KernelContext* ctx, int64_t num_groups) { @@ -813,7 +814,7 @@ struct GroupedMeanImpl (*null_count)++; bit_util::SetBitTo((*null_bitmap)->mutable_data(), i, false); } - return std::move(values); + return values; } std::shared_ptr out_type() const override { @@ -1114,7 +1115,8 @@ Result> VarStdInit(KernelContext* ctx, auto impl = std::make_unique>(); impl->result_type_ = result_type; RETURN_NOT_OK(impl->Init(ctx->exec_context(), args)); - return std::move(impl); + // R build with openSUSE155 requires an explicit unique_ptr construction + return std::unique_ptr(std::move(impl)); } template @@ -1685,7 +1687,7 @@ Result> MinMaxInit(KernelContext* ctx, const KernelInitArgs& args) { ARROW_ASSIGN_OR_RAISE(auto impl, HashAggregateInit>(ctx, args)); static_cast*>(impl.get())->type_ = args.inputs[0].GetSharedPtr(); - return std::move(impl); + return impl; } template @@ -2188,7 +2190,7 @@ Result> FirstLastInit(KernelContext* ctx, ARROW_ASSIGN_OR_RAISE(auto impl, HashAggregateInit>(ctx, args)); static_cast*>(impl.get())->type_ = args.inputs[0].GetSharedPtr(); - return std::move(impl); + return impl; } template @@ -2597,7 +2599,7 @@ Result> GroupedDistinctInit(KernelContext* ctx, instance->out_type_ = args.inputs[0].GetSharedPtr(); ARROW_ASSIGN_OR_RAISE(instance->grouper_, Grouper::Make(args.inputs, ctx->exec_context())); - return std::move(impl); + return impl; } // ---------------------------------------------------------------------- @@ -2839,7 +2841,7 @@ Result> GroupedOneInit(KernelContext* ctx, ARROW_ASSIGN_OR_RAISE(auto impl, HashAggregateInit>(ctx, args)); auto instance = static_cast*>(impl.get()); instance->out_type_ = args.inputs[0].GetSharedPtr(); - return std::move(impl); + return impl; } struct GroupedOneFactory { @@ -3237,7 +3239,7 @@ Result> GroupedListInit(KernelContext* ctx, ARROW_ASSIGN_OR_RAISE(auto impl, HashAggregateInit>(ctx, args)); auto instance = static_cast*>(impl.get()); instance->out_type_ = args.inputs[0].GetSharedPtr(); - return std::move(impl); + return impl; } struct GroupedListFactory { diff --git a/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc b/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc index efd25a8a20c80..eb243de4a765e 100644 --- a/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc +++ b/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc @@ -503,7 +503,7 @@ Result ResolveDecimalBinaryOperationOutput( ToResult(getter(left_type.precision(), left_type.scale(), right_type.precision(), right_type.scale()))); ARROW_ASSIGN_OR_RAISE(auto type, DecimalType::Make(left_type.id(), precision, scale)); - return std::move(type); + return type; } Result ResolveDecimalAdditionOrSubtractionOutput( @@ -566,7 +566,7 @@ Result ResolveTemporalOutput(KernelContext*, } auto type = duration(right_type.unit()); - return std::move(type); + return type; } template diff --git a/cpp/src/arrow/compute/kernels/scalar_string_ascii.cc b/cpp/src/arrow/compute/kernels/scalar_string_ascii.cc index 038e623b43c53..762b666c6a148 100644 --- a/cpp/src/arrow/compute/kernels/scalar_string_ascii.cc +++ b/cpp/src/arrow/compute/kernels/scalar_string_ascii.cc @@ -1315,7 +1315,7 @@ struct RegexSubstringMatcher { const MatchSubstringOptions& options, bool is_utf8 = true, bool literal = false) { auto matcher = std::make_unique(options, is_utf8, literal); RETURN_NOT_OK(RegexStatus(matcher->regex_match_)); - return std::move(matcher); + return matcher; } explicit RegexSubstringMatcher(const MatchSubstringOptions& options, @@ -1685,7 +1685,7 @@ struct FindSubstringRegex { bool is_utf8 = true, bool literal = false) { auto matcher = FindSubstringRegex(options, is_utf8, literal); RETURN_NOT_OK(RegexStatus(*matcher.regex_match_)); - return std::move(matcher); + return matcher; } explicit FindSubstringRegex(const MatchSubstringOptions& options, bool is_utf8 = true, @@ -1832,7 +1832,7 @@ struct CountSubstringRegex { bool is_utf8 = true, bool literal = false) { CountSubstringRegex counter(options, is_utf8, literal); RETURN_NOT_OK(RegexStatus(*counter.regex_match_)); - return std::move(counter); + return counter; } template @@ -2055,7 +2055,7 @@ struct RegexSubstringReplacer { std::move(replacement_error)); } - return std::move(replacer); + return replacer; } // Using RE2::FindAndConsume we can only find the pattern if it is a group, therefore @@ -2203,7 +2203,7 @@ struct ExtractRegexData { } data.group_names.emplace_back(item->second); } - return std::move(data); + return data; } Result ResolveOutputType(const std::vector& types) const { diff --git a/cpp/src/arrow/compute/kernels/vector_hash.cc b/cpp/src/arrow/compute/kernels/vector_hash.cc index 800deba3a5ed2..44bb7372c3f68 100644 --- a/cpp/src/arrow/compute/kernels/vector_hash.cc +++ b/cpp/src/arrow/compute/kernels/vector_hash.cc @@ -530,7 +530,8 @@ Result> HashInit(KernelContext* ctx, auto result = std::make_unique(args.inputs[0].GetSharedPtr(), args.options, ctx->memory_pool()); RETURN_NOT_OK(result->Reset()); - return std::move(result); + // R build with openSUSE155 requires an explicit unique_ptr construction + return std::unique_ptr(std::move(result)); } template diff --git a/cpp/src/arrow/compute/row/grouper.cc b/cpp/src/arrow/compute/row/grouper.cc index 3ed5411d0ba02..45b9ad5971e80 100644 --- a/cpp/src/arrow/compute/row/grouper.cc +++ b/cpp/src/arrow/compute/row/grouper.cc @@ -347,7 +347,7 @@ struct GrouperNoKeysImpl : Grouper { } std::shared_ptr array; RETURN_NOT_OK(builder->Finish(&array)); - return std::move(array); + return array; } Status Reset() override { return Status::OK(); } Result Consume(const ExecSpan& batch, int64_t offset, int64_t length) override { @@ -359,7 +359,7 @@ struct GrouperNoKeysImpl : Grouper { auto values = data->GetMutableValues(0); values[0] = 0; ExecBatch out({Datum(data)}, 1); - return std::move(out); + return out; } uint32_t num_groups() const override { return 1; } }; @@ -412,7 +412,7 @@ struct GrouperImpl : public Grouper { return Status::NotImplemented("Keys of type ", *key); } - return std::move(impl); + return impl; } Status Reset() override { @@ -596,7 +596,7 @@ struct GrouperFastImpl : public Grouper { impl->minibatch_hashes_.resize(impl->minibatch_size_max_ + kPaddingForSIMD / sizeof(uint32_t)); - return std::move(impl); + return impl; } Status Reset() override { diff --git a/cpp/src/arrow/csv/reader.cc b/cpp/src/arrow/csv/reader.cc index e981fafe8e780..e3033a81486e8 100644 --- a/cpp/src/arrow/csv/reader.cc +++ b/cpp/src/arrow/csv/reader.cc @@ -1053,8 +1053,8 @@ class AsyncThreadedTableReader auto self = shared_from_this(); return ProcessFirstBuffer().Then([self](const std::shared_ptr& first_buffer) { auto block_generator = ThreadedBlockReader::MakeAsyncIterator( - self->buffer_generator_, MakeChunker(self->parse_options_), - std::move(first_buffer), self->read_options_.skip_rows_after_names); + self->buffer_generator_, MakeChunker(self->parse_options_), first_buffer, + self->read_options_.skip_rows_after_names); std::function block_visitor = [self](CSVBlock maybe_block) -> Status { diff --git a/cpp/src/arrow/dataset/dataset.cc b/cpp/src/arrow/dataset/dataset.cc index 0e7bc3da84235..8a3df182474bf 100644 --- a/cpp/src/arrow/dataset/dataset.cc +++ b/cpp/src/arrow/dataset/dataset.cc @@ -402,7 +402,7 @@ class BasicFragmentEvolution : public FragmentEvolutionStrategy { } return compute::field_ref(FieldRef(std::move(modified_indices))); } - return std::move(expr); + return expr; }, [](compute::Expression expr, compute::Expression* old_expr) { return expr; }); }; diff --git a/cpp/src/arrow/dataset/dataset_writer.cc b/cpp/src/arrow/dataset/dataset_writer.cc index 754386275d60c..c60042dd6fef8 100644 --- a/cpp/src/arrow/dataset/dataset_writer.cc +++ b/cpp/src/arrow/dataset/dataset_writer.cc @@ -408,8 +408,7 @@ class DatasetWriterDirectoryQueue { write_options, writer_state); dir_queue->PrepareDirectory(); ARROW_ASSIGN_OR_RAISE(dir_queue->current_filename_, dir_queue->GetNextFilename()); - // std::move required to make RTools 3.5 mingw compiler happy - return std::move(dir_queue); + return dir_queue; } Status Finish() { diff --git a/cpp/src/arrow/dataset/file_csv.cc b/cpp/src/arrow/dataset/file_csv.cc index 09ab775727c98..6258a674deee5 100644 --- a/cpp/src/arrow/dataset/file_csv.cc +++ b/cpp/src/arrow/dataset/file_csv.cc @@ -106,7 +106,7 @@ class CsvFileScanner : public FragmentScanner { } convert_options.include_columns = std::move(columns); convert_options.column_types = std::move(column_types); - return std::move(convert_options); + return convert_options; } static Future> Make( diff --git a/cpp/src/arrow/dataset/file_parquet.cc b/cpp/src/arrow/dataset/file_parquet.cc index c17ba89be7907..1f8b6cc4882cf 100644 --- a/cpp/src/arrow/dataset/file_parquet.cc +++ b/cpp/src/arrow/dataset/file_parquet.cc @@ -279,7 +279,7 @@ Status ResolveOneFieldRef( // names) based on the dataset schema. Returns `false` if no conversion was needed. Result MaybeConvertFieldRef(FieldRef ref, const Schema& dataset_schema) { if (ARROW_PREDICT_TRUE(ref.IsNameSequence())) { - return std::move(ref); + return ref; } ARROW_ASSIGN_OR_RAISE(auto path, ref.FindOne(dataset_schema)); @@ -504,7 +504,8 @@ Result> ParquetFileFormat::GetReader std::unique_ptr arrow_reader; RETURN_NOT_OK(parquet::arrow::FileReader::Make( options->pool, std::move(reader), std::move(arrow_properties), &arrow_reader)); - return std::move(arrow_reader); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(arrow_reader)); } Future> ParquetFileFormat::GetReaderAsync( @@ -543,7 +544,9 @@ Future> ParquetFileFormat::GetReader reader)), std::move(arrow_properties), &arrow_reader)); - return std::move(arrow_reader); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr( + std::move(arrow_reader)); }, [path = source.path()](const Status& status) -> Result> { diff --git a/cpp/src/arrow/dataset/scan_node.cc b/cpp/src/arrow/dataset/scan_node.cc index c25c5b70ae1ec..4493332111429 100644 --- a/cpp/src/arrow/dataset/scan_node.cc +++ b/cpp/src/arrow/dataset/scan_node.cc @@ -166,7 +166,7 @@ class ScanNode : public acero::ExecNode, public acero::TracedNode { return Status::Invalid("A scan filter must be a boolean expression"); } - return std::move(normalized); + return normalized; } static Result Make(acero::ExecPlan* plan, @@ -334,7 +334,7 @@ class ScanNode : public acero::ExecNode, public acero::TracedNode { extracted.known_values.push_back({i, *maybe_casted}); } } - return std::move(extracted); + return extracted; } Future<> BeginScan(const std::shared_ptr& inspected_fragment) { @@ -427,7 +427,7 @@ class ScanNode : public acero::ExecNode, public acero::TracedNode { /*queue=*/nullptr, [this]() { return output_->InputFinished(this, num_batches_.load()); }); fragment_tasks->AddAsyncGenerator>( - std::move(frag_gen), + frag_gen, [this, fragment_tasks = std::move(fragment_tasks)](const std::shared_ptr& fragment) { fragment_tasks->AddTask(std::make_unique(this, fragment)); diff --git a/cpp/src/arrow/device.cc b/cpp/src/arrow/device.cc index 01a2b8df5398d..c2195e89e75ee 100644 --- a/cpp/src/arrow/device.cc +++ b/cpp/src/arrow/device.cc @@ -215,7 +215,7 @@ Result> CPUMemoryManager::CopyNonOwnedFrom( if (buf.size() > 0) { memcpy(dest->mutable_data(), buf.data(), static_cast(buf.size())); } - return std::move(dest); + return dest; } Result> CPUMemoryManager::ViewBufferFrom( @@ -247,7 +247,7 @@ Result> CPUMemoryManager::CopyNonOwnedTo( if (buf.size() > 0) { memcpy(dest->mutable_data(), buf.data(), static_cast(buf.size())); } - return std::move(dest); + return dest; } Result> CPUMemoryManager::ViewBufferTo( diff --git a/cpp/src/arrow/engine/substrait/expression_internal.cc b/cpp/src/arrow/engine/substrait/expression_internal.cc index 480cf30d3033f..56d7956076bf8 100644 --- a/cpp/src/arrow/engine/substrait/expression_internal.cc +++ b/cpp/src/arrow/engine/substrait/expression_internal.cc @@ -133,7 +133,7 @@ Result DecodeScalarFunction( for (const auto& opt : scalar_fn.options()) { ARROW_RETURN_NOT_OK(DecodeOption(opt, &call)); } - return std::move(call); + return call; } std::string EnumToString(int value, const google::protobuf::EnumDescriptor* descriptor) { @@ -279,7 +279,7 @@ Result FromProto(const substrait::AggregateFunction& func, bool i for (int i = 0; i < func.options_size(); i++) { ARROW_RETURN_NOT_OK(DecodeOption(func.options(i), &call)); } - return std::move(call); + return call; } Result FromProto(const substrait::Expression& expr, @@ -1153,7 +1153,7 @@ Result> ToProto( out->set_allocated_null(type.release()); } - return std::move(out); + return out; } static Status AddChildToReferenceSegment( @@ -1226,7 +1226,7 @@ static Result> MakeDirectReference( auto out = std::make_unique(); out->set_allocated_selection(selection.release()); - return std::move(out); + return out; } // Indexes the given Substrait struct-typed expression or root (if expr is empty) using @@ -1292,7 +1292,7 @@ Result> EncodeSubstraitCa } } - return std::move(scalar_fn); + return scalar_fn; } Result>> DatumToLiterals( @@ -1356,7 +1356,7 @@ Result> ToProto( if (auto datum = expr.literal()) { ARROW_ASSIGN_OR_RAISE(auto literal, ToProto(*datum, ext_set, conversion_options)); out->set_allocated_literal(literal.release()); - return std::move(out); + return out; } if (auto param = expr.parameter()) { @@ -1367,7 +1367,7 @@ Result> ToProto( ARROW_ASSIGN_OR_RAISE(out, MakeStructFieldReference(std::move(out), index)); } - return std::move(out); + return out; } auto call = CallNotNull(expr); @@ -1399,7 +1399,7 @@ Result> ToProto( if_then_->set_allocated_else_(arguments.back().release()); out->set_allocated_if_then(if_then_.release()); - return std::move(out); + return out; } } @@ -1423,7 +1423,7 @@ Result> ToProto( for (int index : field_path.indices()) { ARROW_ASSIGN_OR_RAISE(out, MakeStructFieldReference(std::move(out), index)); } - return std::move(out); + return out; } if (call->function_name == "list_element") { @@ -1449,7 +1449,7 @@ Result> ToProto( if_then->set_allocated_else_(arguments[2].release()); out->set_allocated_if_then(if_then.release()); - return std::move(out); + return out; } else if (call->function_name == "cast") { auto cast = std::make_unique(); @@ -1478,7 +1478,7 @@ Result> ToProto( cast->set_allocated_type(to_type.release()); out->set_allocated_cast(cast.release()); - return std::move(out); + return out; } else if (call->function_name == "is_in") { auto or_list = std::make_unique(); @@ -1499,7 +1499,7 @@ Result> ToProto( or_list->mutable_options()->AddAllocated(option.release()); } out->set_allocated_singular_or_list(or_list.release()); - return std::move(out); + return out; } // other expression types dive into extensions immediately @@ -1534,7 +1534,7 @@ Result> ToProto( return maybe_converter.status(); } out->set_allocated_scalar_function(scalar_fn.release()); - return std::move(out); + return out; } } // namespace engine diff --git a/cpp/src/arrow/engine/substrait/extended_expression_internal.cc b/cpp/src/arrow/engine/substrait/extended_expression_internal.cc index 225901c910f25..e2e6d934372dc 100644 --- a/cpp/src/arrow/engine/substrait/extended_expression_internal.cc +++ b/cpp/src/arrow/engine/substrait/extended_expression_internal.cc @@ -143,7 +143,7 @@ Result> CreateExpressionReferenc ARROW_ASSIGN_OR_RAISE(std::unique_ptr expression, ToProto(expr, ext_set, conversion_options)); expr_ref->set_allocated_expression(expression.release()); - return std::move(expr_ref); + return expr_ref; } } // namespace @@ -178,7 +178,7 @@ Result FromProto(const substrait::ExtendedExpression& expressi *ext_set_out = std::move(ext_set); } - return std::move(bound_expressions); + return bound_expressions; } Result> ToProto( @@ -203,7 +203,7 @@ Result> ToProto( expression->mutable_referred_expr()->AddAllocated(expr_ref.release()); } RETURN_NOT_OK(AddExtensionSetToExtendedExpression(*ext_set, expression.get())); - return std::move(expression); + return expression; } } // namespace engine diff --git a/cpp/src/arrow/engine/substrait/extension_set.cc b/cpp/src/arrow/engine/substrait/extension_set.cc index e955084dcdfbb..cefe53d2847ca 100644 --- a/cpp/src/arrow/engine/substrait/extension_set.cc +++ b/cpp/src/arrow/engine/substrait/extension_set.cc @@ -288,7 +288,7 @@ Result ExtensionSet::Make( } } - return std::move(set); + return set; } Result ExtensionSet::DecodeType(uint32_t anchor) const { @@ -799,7 +799,7 @@ Result> GetValueArgs(const SubstraitCall& call, ARROW_ASSIGN_OR_RAISE(compute::Expression arg, call.GetValueArg(index)); expressions.push_back(arg); } - return std::move(expressions); + return expressions; } ExtensionIdRegistry::SubstraitCallToArrow DecodeOptionlessOverflowableArithmetic( @@ -874,7 +874,7 @@ ExtensionIdRegistry::ArrowToSubstraitCall EncodeOptionlessOverflowableArithmetic for (std::size_t i = 0; i < call.arguments.size(); i++) { substrait_call.SetValueArg(static_cast(i), call.arguments[i]); } - return std::move(substrait_call); + return substrait_call; }; } @@ -887,7 +887,7 @@ ExtensionIdRegistry::ArrowToSubstraitCall EncodeBasic(Id substrait_fn_id) { for (std::size_t i = 0; i < call.arguments.size(); i++) { substrait_call.SetValueArg(static_cast(i), call.arguments[i]); } - return std::move(substrait_call); + return substrait_call; }; } @@ -907,7 +907,7 @@ ExtensionIdRegistry::ArrowToSubstraitCall EncodeIsNull(Id substrait_fn_id) { for (std::size_t i = 0; i < call.arguments.size(); i++) { substrait_call.SetValueArg(static_cast(i), call.arguments[i]); } - return std::move(substrait_call); + return substrait_call; }; } diff --git a/cpp/src/arrow/engine/substrait/plan_internal.cc b/cpp/src/arrow/engine/substrait/plan_internal.cc index cc4806878c404..4473b0443eb19 100644 --- a/cpp/src/arrow/engine/substrait/plan_internal.cc +++ b/cpp/src/arrow/engine/substrait/plan_internal.cc @@ -65,7 +65,7 @@ Result> PlanToProto( plan_rel->set_allocated_root(rel_root.release()); subs_plan->mutable_relations()->AddAllocated(plan_rel.release()); RETURN_NOT_OK(AddExtensionSetToPlan(*ext_set, subs_plan.get())); - return std::move(subs_plan); + return subs_plan; } } // namespace engine diff --git a/cpp/src/arrow/engine/substrait/relation_internal.cc b/cpp/src/arrow/engine/substrait/relation_internal.cc index 7c462c418f81b..6a25bd89f0128 100644 --- a/cpp/src/arrow/engine/substrait/relation_internal.cc +++ b/cpp/src/arrow/engine/substrait/relation_internal.cc @@ -91,7 +91,7 @@ Result GetEmitInfo(const RelMessage& rel, } emit_info.expressions = std::move(proj_field_refs); emit_info.schema = schema(std::move(emit_fields)); - return std::move(emit_info); + return emit_info; } Result ProcessEmitProject( @@ -1024,7 +1024,7 @@ Result> NamedTableRelationConverter( } read_rel->set_allocated_named_table(read_rel_tn.release()); - return std::move(read_rel); + return read_rel; } Result> ScanRelationConverter( @@ -1068,7 +1068,7 @@ Result> ScanRelationConverter( read_rel_lfs->mutable_items()->AddAllocated(read_rel_lfs_ffs.release()); } read_rel->set_allocated_local_files(read_rel_lfs.release()); - return std::move(read_rel); + return read_rel; } Result> FilterRelationConverter( @@ -1097,7 +1097,7 @@ Result> FilterRelationConverter( ARROW_ASSIGN_OR_RAISE(auto subs_expr, ToProto(bound_expression, ext_set, conversion_options)); filter_rel->set_allocated_condition(subs_expr.release()); - return std::move(filter_rel); + return filter_rel; } } // namespace @@ -1146,7 +1146,7 @@ Result> ToProto( const ConversionOptions& conversion_options) { auto rel = std::make_unique(); RETURN_NOT_OK(SerializeAndCombineRelations(declr, ext_set, &rel, conversion_options)); - return std::move(rel); + return rel; } } // namespace engine diff --git a/cpp/src/arrow/engine/substrait/serde.cc b/cpp/src/arrow/engine/substrait/serde.cc index 9e670f121778e..16d2ace4ac0d7 100644 --- a/cpp/src/arrow/engine/substrait/serde.cc +++ b/cpp/src/arrow/engine/substrait/serde.cc @@ -256,7 +256,7 @@ Result> MakeSingleDeclarationPlan( } else { ARROW_ASSIGN_OR_RAISE(auto plan, acero::ExecPlan::Make()); ARROW_RETURN_NOT_OK(declarations[0].AddToPlan(plan.get())); - return std::move(plan); + return plan; } } diff --git a/cpp/src/arrow/engine/substrait/type_internal.cc b/cpp/src/arrow/engine/substrait/type_internal.cc index 5e7e364fe00c5..b469f5fa0baab 100644 --- a/cpp/src/arrow/engine/substrait/type_internal.cc +++ b/cpp/src/arrow/engine/substrait/type_internal.cc @@ -506,7 +506,7 @@ Result> ToProto( auto out = std::make_unique(); RETURN_NOT_OK( (DataTypeToProtoImpl{out.get(), nullable, ext_set, conversion_options})(type)); - return std::move(out); + return out; } Result> FromProto(const substrait::NamedStruct& named_struct, @@ -583,7 +583,7 @@ Result> ToProto( } named_struct->set_allocated_struct_(struct_.release()); - return std::move(named_struct); + return named_struct; } } // namespace engine diff --git a/cpp/src/arrow/field_ref_test.cc b/cpp/src/arrow/field_ref_test.cc index 0cb2da4f709a1..25c8aa0b71623 100644 --- a/cpp/src/arrow/field_ref_test.cc +++ b/cpp/src/arrow/field_ref_test.cc @@ -199,7 +199,7 @@ struct FieldPathTestCase { Table::Make(out.schema, {out.v0.chunked_array, out.v1.chunked_array}, kNumRows); ARROW_RETURN_NOT_OK(out.table->ValidateFull()); - return std::move(out); + return out; } private: diff --git a/cpp/src/arrow/filesystem/azurefs.cc b/cpp/src/arrow/filesystem/azurefs.cc index ac563b134586e..7462827d80f1e 100644 --- a/cpp/src/arrow/filesystem/azurefs.cc +++ b/cpp/src/arrow/filesystem/azurefs.cc @@ -845,7 +845,7 @@ class ObjectInputFile final : public io::RandomAccessFile { DCHECK_LE(bytes_read, nbytes); RETURN_NOT_OK(buffer->Resize(bytes_read)); } - return std::move(buffer); + return buffer; } Result Read(int64_t nbytes, void* out) override { @@ -857,7 +857,7 @@ class ObjectInputFile final : public io::RandomAccessFile { Result> Read(int64_t nbytes) override { ARROW_ASSIGN_OR_RAISE(auto buffer, ReadAt(pos_, nbytes)); pos_ += buffer->size(); - return std::move(buffer); + return buffer; } private: diff --git a/cpp/src/arrow/filesystem/localfs.cc b/cpp/src/arrow/filesystem/localfs.cc index 25ac04b758f9b..22d802d8f9f7f 100644 --- a/cpp/src/arrow/filesystem/localfs.cc +++ b/cpp/src/arrow/filesystem/localfs.cc @@ -524,7 +524,7 @@ class AsyncStatSelector { ARROW_ASSIGN_OR_RAISE( auto gen, MakeBackgroundGenerator(Iterator(DiscoveryImplIterator( - std::move(dir_fn), nesting_depth, std::move(selector), + dir_fn, nesting_depth, std::move(selector), discovery_state, io_context, file_info_batch_size)), io_context.executor())); gen = MakeTransferredGenerator(std::move(gen), io_context.executor()); diff --git a/cpp/src/arrow/filesystem/localfs_test.cc b/cpp/src/arrow/filesystem/localfs_test.cc index d68c992dff863..6dd7a8c75586c 100644 --- a/cpp/src/arrow/filesystem/localfs_test.cc +++ b/cpp/src/arrow/filesystem/localfs_test.cc @@ -543,7 +543,7 @@ struct DirTreeCreator { Result Create(const std::string& base) { FileInfoVector infos; RETURN_NOT_OK(Create(base, 0, &infos)); - return std::move(infos); + return infos; } Status Create(const std::string& base, int depth, FileInfoVector* infos) { diff --git a/cpp/src/arrow/filesystem/s3fs.cc b/cpp/src/arrow/filesystem/s3fs.cc index 640888e1c4fa5..43666f32b3da6 100644 --- a/cpp/src/arrow/filesystem/s3fs.cc +++ b/cpp/src/arrow/filesystem/s3fs.cc @@ -1510,7 +1510,8 @@ class ObjectInputFile final : public io::RandomAccessFile { DCHECK_LE(bytes_read, nbytes); RETURN_NOT_OK(buf->Resize(bytes_read)); } - return std::move(buf); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(buf)); } Result Read(int64_t nbytes, void* out) override { @@ -1522,7 +1523,7 @@ class ObjectInputFile final : public io::RandomAccessFile { Result> Read(int64_t nbytes) override { ARROW_ASSIGN_OR_RAISE(auto buffer, ReadAt(pos_, nbytes)); pos_ += buffer->size(); - return std::move(buffer); + return buffer; } protected: diff --git a/cpp/src/arrow/filesystem/util_internal.cc b/cpp/src/arrow/filesystem/util_internal.cc index d69f6c896d08e..be43e14e84337 100644 --- a/cpp/src/arrow/filesystem/util_internal.cc +++ b/cpp/src/arrow/filesystem/util_internal.cc @@ -103,7 +103,7 @@ Result ParseFileSystemUri(const std::string& uri_string) { return status; #endif } - return std::move(uri); + return uri; } #ifdef _WIN32 diff --git a/cpp/src/arrow/flight/client.cc b/cpp/src/arrow/flight/client.cc index 4d4f13a09fb26..58a3ba4ab83e5 100644 --- a/cpp/src/arrow/flight/client.cc +++ b/cpp/src/arrow/flight/client.cc @@ -591,7 +591,7 @@ arrow::Result FlightClient::CancelFlightInfo( ARROW_ASSIGN_OR_RAISE(auto cancel_result, CancelFlightInfoResult::Deserialize( std::string_view(*result->body))); ARROW_RETURN_NOT_OK(stream->Drain()); - return std::move(cancel_result); + return cancel_result; } arrow::Result FlightClient::RenewFlightEndpoint( @@ -603,7 +603,7 @@ arrow::Result FlightClient::RenewFlightEndpoint( ARROW_ASSIGN_OR_RAISE(auto renewed_endpoint, FlightEndpoint::Deserialize(std::string_view(*result->body))); ARROW_RETURN_NOT_OK(stream->Drain()); - return std::move(renewed_endpoint); + return renewed_endpoint; } arrow::Result> FlightClient::ListActions( diff --git a/cpp/src/arrow/flight/sql/client.cc b/cpp/src/arrow/flight/sql/client.cc index 37b6a0b32e45d..86fd4868bad2d 100644 --- a/cpp/src/arrow/flight/sql/client.cc +++ b/cpp/src/arrow/flight/sql/client.cc @@ -682,7 +682,7 @@ arrow::Result> PreparedStatement::Execute( parameter_binding_.get())); } ARROW_ASSIGN_OR_RAISE(auto flight_info, client_->GetFlightInfo(options, descriptor)); - return std::move(flight_info); + return flight_info; } arrow::Result PreparedStatement::ExecuteUpdate( diff --git a/cpp/src/arrow/gpu/cuda_memory.cc b/cpp/src/arrow/gpu/cuda_memory.cc index dcf0a31963e45..148de68434272 100644 --- a/cpp/src/arrow/gpu/cuda_memory.cc +++ b/cpp/src/arrow/gpu/cuda_memory.cc @@ -89,7 +89,7 @@ Result> CudaIpcMemHandle::Serialize(MemoryPool* pool) co memcpy(buffer->mutable_data() + sizeof(impl_->memory_size), &impl_->ipc_handle, sizeof(impl_->ipc_handle)); } - return std::move(buffer); + return buffer; } const void* CudaIpcMemHandle::handle() const { return &impl_->ipc_handle; } diff --git a/cpp/src/arrow/integration/json_internal.cc b/cpp/src/arrow/integration/json_internal.cc index 4b75e84bfccb6..89719b4ba4b2e 100644 --- a/cpp/src/arrow/integration/json_internal.cc +++ b/cpp/src/arrow/integration/json_internal.cc @@ -1069,9 +1069,9 @@ Result> GetUnion(const RjObject& json_type, } if (mode == UnionMode::SPARSE) { - return sparse_union(std::move(children), std::move(type_codes)); + return sparse_union(children, std::move(type_codes)); } else { - return dense_union(std::move(children), std::move(type_codes)); + return dense_union(children, std::move(type_codes)); } } diff --git a/cpp/src/arrow/io/buffered.cc b/cpp/src/arrow/io/buffered.cc index 21cce478d3fa5..c53b3d223d4c0 100644 --- a/cpp/src/arrow/io/buffered.cc +++ b/cpp/src/arrow/io/buffered.cc @@ -423,7 +423,8 @@ class BufferedInputStream::Impl : public BufferedBase { RETURN_NOT_OK(buffer->Resize(bytes_read, false /* shrink_to_fit */)); buffer->ZeroPadding(); } - return std::move(buffer); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(buffer)); } // For providing access to the raw file handles diff --git a/cpp/src/arrow/io/compressed.cc b/cpp/src/arrow/io/compressed.cc index 6a6fbf40f9628..306f7c4bf8519 100644 --- a/cpp/src/arrow/io/compressed.cc +++ b/cpp/src/arrow/io/compressed.cc @@ -411,9 +411,8 @@ class CompressedInputStream::Impl { ARROW_ASSIGN_OR_RAISE(auto buf, AllocateResizableBuffer(nbytes, pool_)); ARROW_ASSIGN_OR_RAISE(int64_t bytes_read, Read(nbytes, buf->mutable_data())); RETURN_NOT_OK(buf->Resize(bytes_read)); - // Using std::move because some compiler might has issue below: - // https://wg21.cmeerw.net/cwg/issue1579 - return std::move(buf); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(buf)); } const std::shared_ptr& raw() const { return raw_; } diff --git a/cpp/src/arrow/io/compressed_test.cc b/cpp/src/arrow/io/compressed_test.cc index bd414149d5345..12d116e3395d4 100644 --- a/cpp/src/arrow/io/compressed_test.cc +++ b/cpp/src/arrow/io/compressed_test.cc @@ -77,7 +77,7 @@ std::shared_ptr CompressDataOneShot(Codec* codec, compressed_len = *codec->Compress(data.size(), data.data(), max_compressed_len, compressed->mutable_data()); ABORT_NOT_OK(compressed->Resize(compressed_len)); - return std::move(compressed); + return compressed; } Status RunCompressedInputStream(Codec* codec, std::shared_ptr compressed, diff --git a/cpp/src/arrow/io/file.cc b/cpp/src/arrow/io/file.cc index cc3a5187059e9..a22accf65660a 100644 --- a/cpp/src/arrow/io/file.cc +++ b/cpp/src/arrow/io/file.cc @@ -230,7 +230,8 @@ class ReadableFile::ReadableFileImpl : public OSFile { RETURN_NOT_OK(buffer->Resize(bytes_read)); buffer->ZeroPadding(); } - return std::move(buffer); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(buffer)); } Result> ReadBufferAt(int64_t position, int64_t nbytes) { @@ -242,7 +243,8 @@ class ReadableFile::ReadableFileImpl : public OSFile { RETURN_NOT_OK(buffer->Resize(bytes_read)); buffer->ZeroPadding(); } - return std::move(buffer); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(buffer)); } Status WillNeed(const std::vector& ranges) { diff --git a/cpp/src/arrow/io/hdfs.cc b/cpp/src/arrow/io/hdfs.cc index 5d3edcd3ba63a..5fb762d076376 100644 --- a/cpp/src/arrow/io/hdfs.cc +++ b/cpp/src/arrow/io/hdfs.cc @@ -172,7 +172,8 @@ class HdfsReadableFile::HdfsReadableFileImpl : public HdfsAnyFileImpl { RETURN_NOT_OK(buffer->Resize(bytes_read)); buffer->ZeroPadding(); } - return std::move(buffer); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(buffer)); } Result Read(int64_t nbytes, void* buffer) { @@ -200,7 +201,8 @@ class HdfsReadableFile::HdfsReadableFileImpl : public HdfsAnyFileImpl { if (bytes_read < nbytes) { RETURN_NOT_OK(buffer->Resize(bytes_read)); } - return std::move(buffer); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(buffer)); } Result GetSize() { diff --git a/cpp/src/arrow/io/stdio.cc b/cpp/src/arrow/io/stdio.cc index ac6ca3a63a604..ba4a66a2f340e 100644 --- a/cpp/src/arrow/io/stdio.cc +++ b/cpp/src/arrow/io/stdio.cc @@ -85,7 +85,8 @@ Result> StdinStream::Read(int64_t nbytes) { ARROW_ASSIGN_OR_RAISE(int64_t bytes_read, Read(nbytes, buffer->mutable_data())); ARROW_RETURN_NOT_OK(buffer->Resize(bytes_read, false)); buffer->ZeroPadding(); - return std::move(buffer); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(buffer)); } } // namespace io diff --git a/cpp/src/arrow/ipc/message.cc b/cpp/src/arrow/ipc/message.cc index e196dd7bf5389..27ded52861eaf 100644 --- a/cpp/src/arrow/ipc/message.cc +++ b/cpp/src/arrow/ipc/message.cc @@ -130,7 +130,7 @@ Result> Message::Open(std::shared_ptr metadata, std::shared_ptr body) { std::unique_ptr result(new Message(std::move(metadata), std::move(body))); RETURN_NOT_OK(result->impl_->Open()); - return std::move(result); + return result; } Message::~Message() {} @@ -208,7 +208,7 @@ Result> Message::ReadFrom(std::shared_ptr metad " bytes for message body, got ", body->size()); } RETURN_NOT_OK(decoder.Consume(body)); - return std::move(result); + return result; } Result> Message::ReadFrom(const int64_t offset, @@ -225,7 +225,7 @@ Result> Message::ReadFrom(const int64_t offset, " bytes for message body, got ", body->size()); } RETURN_NOT_OK(decoder.Consume(body)); - return std::move(result); + return result; } Status WritePadding(io::OutputStream* stream, int64_t nbytes) { @@ -329,7 +329,7 @@ Result> ReadMessage(std::shared_ptr metadata, case MessageDecoder::State::INITIAL: // Metadata did not request a body so we better not have provided one DCHECK_EQ(body, nullptr); - return std::move(result); + return result; case MessageDecoder::State::METADATA_LENGTH: return Status::Invalid("metadata length is missing from the metadata buffer"); case MessageDecoder::State::METADATA: @@ -338,7 +338,7 @@ Result> ReadMessage(std::shared_ptr metadata, case MessageDecoder::State::BODY: { if (body == nullptr) { // Caller didn't give a body so just give them a message without body - return std::move(result); + return result; } if (body->size() != decoder.next_required_size()) { return Status::IOError("Expected body buffer to be ", @@ -346,7 +346,7 @@ Result> ReadMessage(std::shared_ptr metadata, " bytes for message body, got ", body->size()); } RETURN_NOT_OK(decoder.Consume(body)); - return std::move(result); + return result; } case MessageDecoder::State::EOS: return Status::Invalid("Unexpected empty message in IPC file format"); @@ -376,7 +376,7 @@ Result> ReadMessage(int64_t offset, int32_t metadata_le switch (decoder.state()) { case MessageDecoder::State::INITIAL: - return std::move(result); + return result; case MessageDecoder::State::METADATA_LENGTH: return Status::Invalid("metadata length is missing. File offset: ", offset, ", metadata length: ", metadata_length); @@ -401,7 +401,7 @@ Result> ReadMessage(int64_t offset, int32_t metadata_le " bytes for message body, got ", body->size()); } RETURN_NOT_OK(decoder.Consume(body)); - return std::move(result); + return result; } case MessageDecoder::State::EOS: return Status::Invalid("Unexpected empty message in IPC file format"); @@ -551,7 +551,7 @@ Result> ReadMessage(io::InputStream* file, MemoryPool* if (!message) { return nullptr; } else { - return std::move(message); + return message; } } diff --git a/cpp/src/arrow/ipc/metadata_internal.cc b/cpp/src/arrow/ipc/metadata_internal.cc index eed426d9337dd..be8d1ccc35f1a 100644 --- a/cpp/src/arrow/ipc/metadata_internal.cc +++ b/cpp/src/arrow/ipc/metadata_internal.cc @@ -191,11 +191,9 @@ Status UnionFromFlatbuffer(const flatbuf::Union* union_data, } if (mode == UnionMode::SPARSE) { - ARROW_ASSIGN_OR_RAISE( - *out, SparseUnionType::Make(std::move(children), std::move(type_codes))); + ARROW_ASSIGN_OR_RAISE(*out, SparseUnionType::Make(children, std::move(type_codes))); } else { - ARROW_ASSIGN_OR_RAISE( - *out, DenseUnionType::Make(std::move(children), std::move(type_codes))); + ARROW_ASSIGN_OR_RAISE(*out, DenseUnionType::Make(children, std::move(type_codes))); } return Status::OK(); } diff --git a/cpp/src/arrow/ipc/metadata_internal.h b/cpp/src/arrow/ipc/metadata_internal.h index 631a336f75a9a..c0aca44644a40 100644 --- a/cpp/src/arrow/ipc/metadata_internal.h +++ b/cpp/src/arrow/ipc/metadata_internal.h @@ -238,7 +238,8 @@ static inline Result> WriteFlatbufferBuilder( uint8_t* dst = result->mutable_data(); memcpy(dst, fbb.GetBufferPointer(), size); - return std::move(result); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(result)); } ARROW_EXPORT diff --git a/cpp/src/arrow/ipc/reader.cc b/cpp/src/arrow/ipc/reader.cc index d272c78560f82..da84f2f2dc87d 100644 --- a/cpp/src/arrow/ipc/reader.cc +++ b/cpp/src/arrow/ipc/reader.cc @@ -540,7 +540,8 @@ Result> DecompressBuffer(const std::shared_ptr& actual_decompressed); } - return std::move(uncompressed); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(uncompressed)); } Status DecompressBuffers(Compression::type compression, const IpcReadOptions& options, @@ -1174,7 +1175,7 @@ static Result> ReadMessageFromBlock( ARROW_ASSIGN_OR_RAISE(auto message, ReadMessage(block.offset, block.metadata_length, file, fields_loader)); - return std::move(message); + return message; } static Future> ReadMessageFromBlockAsync( @@ -1536,7 +1537,7 @@ class RecordBatchFileReaderImpl : public RecordBatchFileReader { ARROW_ASSIGN_OR_RAISE(auto message, arrow::ipc::ReadMessageFromBlock(block, file_, fields_loader)); stats_.num_messages.fetch_add(1, std::memory_order_relaxed); - return std::move(message); + return message; } Status ReadDictionaries() { @@ -1632,7 +1633,7 @@ class RecordBatchFileReaderImpl : public RecordBatchFileReader { } context.compression = compression; context.metadata_version = internal::GetMetadataVersion(message->version()); - return std::move(context); + return context; } Result GetBatchFromMessage( @@ -2704,7 +2705,7 @@ Result> IoRecordedRandomAccessFile::Read(int64_t nbytes) ARROW_ASSIGN_OR_RAISE(std::shared_ptr buffer, ReadAt(position_, nbytes)); auto num_bytes_read = std::min(file_size_, position_ + nbytes) - position_; position_ += num_bytes_read; - return std::move(buffer); + return buffer; } const io::IOContext& IoRecordedRandomAccessFile::io_context() const { diff --git a/cpp/src/arrow/ipc/writer.cc b/cpp/src/arrow/ipc/writer.cc index 93256440f4a7a..e3dd36efe0543 100644 --- a/cpp/src/arrow/ipc/writer.cc +++ b/cpp/src/arrow/ipc/writer.cc @@ -1561,7 +1561,8 @@ Result> OpenRecordBatchWriter( auto writer = std::make_unique( std::move(sink), schema, options, /*is_file_format=*/false); RETURN_NOT_OK(writer->Start()); - return std::move(writer); + // R build with openSUSE155 requires an explicit unique_ptr construction + return std::unique_ptr(std::move(writer)); } Result> MakePayloadStreamWriter( diff --git a/cpp/src/arrow/scalar.cc b/cpp/src/arrow/scalar.cc index 7d8084e17c279..252706fd0b387 100644 --- a/cpp/src/arrow/scalar.cc +++ b/cpp/src/arrow/scalar.cc @@ -1223,7 +1223,7 @@ Result> CastImpl(const StringScalar& from, ARROW_ASSIGN_OR_RAISE(auto out, Scalar::Parse(std::move(to_type), std::string_view(*from.value))); DCHECK(checked_pointer_cast(out) != nullptr); - return std::move(out); + return out; } // binary/large binary/large string to string @@ -1347,7 +1347,7 @@ struct FromTypeVisitor : CastImplVisitor { ARROW_ASSIGN_OR_RAISE( out_, CastImpl( checked_cast::ScalarType&>(from_), - std::move(to_type_))); + to_type_)); return Status::OK(); } @@ -1355,8 +1355,8 @@ struct FromTypeVisitor : CastImplVisitor { template typename std::enable_if_t::is_parameter_free, Status> Visit( const ToType&) { - ARROW_ASSIGN_OR_RAISE(out_, MakeScalar(std::move(to_type_), - checked_cast(from_).value)); + ARROW_ASSIGN_OR_RAISE( + out_, MakeScalar(to_type_, checked_cast(from_).value)); return Status::OK(); } diff --git a/cpp/src/arrow/table_builder.cc b/cpp/src/arrow/table_builder.cc index 19ca151ac200f..8dc2efd19d90d 100644 --- a/cpp/src/arrow/table_builder.cc +++ b/cpp/src/arrow/table_builder.cc @@ -47,7 +47,7 @@ Result> RecordBatchBuilder::Make( new RecordBatchBuilder(schema, pool, initial_capacity)); RETURN_NOT_OK(builder->CreateBuilders()); RETURN_NOT_OK(builder->InitBuilders()); - return std::move(builder); + return builder; } Result> RecordBatchBuilder::Flush(bool reset_builders) { diff --git a/cpp/src/arrow/testing/builder.h b/cpp/src/arrow/testing/builder.h index 09e8f49dea9eb..6beb7760e3bbf 100644 --- a/cpp/src/arrow/testing/builder.h +++ b/cpp/src/arrow/testing/builder.h @@ -195,7 +195,7 @@ Result> ArrayFromBuilderVisitor( std::shared_ptr out; RETURN_NOT_OK(builder->Finish(&out)); - return std::move(out); + return out; } template diff --git a/cpp/src/arrow/type.cc b/cpp/src/arrow/type.cc index edf8f0496628c..8ce03a91c70ae 100644 --- a/cpp/src/arrow/type.cc +++ b/cpp/src/arrow/type.cc @@ -729,7 +729,7 @@ Result> MaybeMergeListTypes( auto item_field, left.item_field()->MergeWith( *right.item_field()->WithName(left.item_field()->name()), options)); - return map(std::move(key_field->type()), std::move(item_field), + return map(key_field->type(), std::move(item_field), /*keys_sorted=*/left.keys_sorted() && right.keys_sorted()); } else if (promoted_type->id() == Type::STRUCT && other_type->id() == Type::STRUCT) { return MergeStructs(promoted_type, other_type, options); @@ -1696,7 +1696,7 @@ class NestedSelector { } } - return std::move(child_data); + return child_data; } static Result> GetChild(const Array& array, int i, diff --git a/cpp/src/arrow/util/align_util.cc b/cpp/src/arrow/util/align_util.cc index 7bc687b155052..a327afa7a5cc3 100644 --- a/cpp/src/arrow/util/align_util.cc +++ b/cpp/src/arrow/util/align_util.cc @@ -159,9 +159,10 @@ Result> EnsureAlignment(std::shared_ptr buffer, auto new_buffer, AllocateBuffer(buffer->size(), minimum_desired_alignment, memory_pool)); std::memcpy(new_buffer->mutable_data(), buffer->data(), buffer->size()); - return std::move(new_buffer); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(new_buffer)); } else { - return std::move(buffer); + return buffer; } } @@ -197,9 +198,9 @@ Result> EnsureAlignment(std::shared_ptr ar auto new_array_data = ArrayData::Make( array_data->type, array_data->length, std::move(buffers), array_data->child_data, array_data->dictionary, array_data->GetNullCount(), array_data->offset); - return std::move(new_array_data); + return new_array_data; } else { - return std::move(array_data); + return array_data; } } @@ -210,7 +211,7 @@ Result> EnsureAlignment(std::shared_ptr array, EnsureAlignment(array->data(), alignment, memory_pool)); if (new_array_data.get() == array->data().get()) { - return std::move(array); + return array; } else { return MakeArray(std::move(new_array_data)); } @@ -230,7 +231,7 @@ Result> EnsureAlignment(std::shared_ptrtype()); } else { - return std::move(array); + return array; } } @@ -248,7 +249,7 @@ Result> EnsureAlignment(std::shared_ptrschema(), batch->num_rows(), std::move(columns_)); } else { - return std::move(batch); + return batch; } } @@ -275,7 +276,7 @@ Result> EnsureAlignment(std::shared_ptr table, } return Table::Make(table->schema(), std::move(columns_), table->num_rows()); } else { - return std::move(table); + return table; } } diff --git a/cpp/src/arrow/util/async_generator.h b/cpp/src/arrow/util/async_generator.h index f9bcd534567c6..fd66298d1a9d6 100644 --- a/cpp/src/arrow/util/async_generator.h +++ b/cpp/src/arrow/util/async_generator.h @@ -1962,7 +1962,7 @@ AsyncGenerator MakeFailingGenerator(Status st) { return [state]() -> Future { auto st = std::move(*state); if (!st.ok()) { - return std::move(st); + return st; } else { return AsyncGeneratorEnd(); } diff --git a/cpp/src/arrow/util/bit_util_benchmark.cc b/cpp/src/arrow/util/bit_util_benchmark.cc index 0bf2c26f12486..43f3fb33cd7fd 100644 --- a/cpp/src/arrow/util/bit_util_benchmark.cc +++ b/cpp/src/arrow/util/bit_util_benchmark.cc @@ -107,7 +107,7 @@ static std::shared_ptr CreateRandomBuffer(int64_t nbytes) { auto buffer = *AllocateBuffer(nbytes); memset(buffer->mutable_data(), 0, nbytes); random_bytes(nbytes, /*seed=*/0, buffer->mutable_data()); - return std::move(buffer); + return buffer; } static std::shared_ptr CreateRandomBitsBuffer(int64_t nbits, diff --git a/cpp/src/arrow/util/bitmap_builders.cc b/cpp/src/arrow/util/bitmap_builders.cc index 0348b1303b96b..c5cf3d2bc72b5 100644 --- a/cpp/src/arrow/util/bitmap_builders.cc +++ b/cpp/src/arrow/util/bitmap_builders.cc @@ -51,7 +51,8 @@ Result> BytesToBits(const std::vector& bytes, uint8_t* out_buf = buffer->mutable_data(); memset(out_buf, 0, static_cast(buffer->capacity())); FillBitsFromBytes(bytes, out_buf); - return std::move(buffer); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(buffer)); } Result> BitmapAllButOne(MemoryPool* pool, int64_t length, @@ -66,7 +67,8 @@ Result> BitmapAllButOne(MemoryPool* pool, int64_t length auto bitmap_data = buffer->mutable_data(); bit_util::SetBitsTo(bitmap_data, 0, length, value); bit_util::SetBitTo(bitmap_data, straggler_pos, !value); - return std::move(buffer); + // R build with openSUSE155 requires an explicit shared_ptr construction + return std::shared_ptr(std::move(buffer)); } } // namespace internal diff --git a/cpp/src/arrow/util/bitmap_reader_benchmark.cc b/cpp/src/arrow/util/bitmap_reader_benchmark.cc index 1427adb13e131..b3c199ec3bd55 100644 --- a/cpp/src/arrow/util/bitmap_reader_benchmark.cc +++ b/cpp/src/arrow/util/bitmap_reader_benchmark.cc @@ -45,7 +45,7 @@ static std::shared_ptr CreateRandomBuffer(int64_t nbytes) { auto buffer = *AllocateBuffer(nbytes); memset(buffer->mutable_data(), 0, nbytes); random_bytes(nbytes, /*seed=*/0, buffer->mutable_data()); - return std::move(buffer); + return buffer; } static void BitBlockCounterBench(benchmark::State& state) { diff --git a/cpp/src/arrow/util/compression.cc b/cpp/src/arrow/util/compression.cc index b63aec0aae8f9..7e2a3de30306a 100644 --- a/cpp/src/arrow/util/compression.cc +++ b/cpp/src/arrow/util/compression.cc @@ -216,7 +216,7 @@ Result> Codec::Create(Compression::type codec_type, DCHECK_NE(codec, nullptr); RETURN_NOT_OK(codec->Init()); - return std::move(codec); + return codec; } // use compression level to create Codec diff --git a/cpp/src/arrow/util/decimal.cc b/cpp/src/arrow/util/decimal.cc index ce71def497161..c8457eae8ed33 100644 --- a/cpp/src/arrow/util/decimal.cc +++ b/cpp/src/arrow/util/decimal.cc @@ -717,7 +717,7 @@ Status Decimal128::FromString(const char* s, Decimal128* out, int32_t* precision Result Decimal128::FromString(std::string_view s) { Decimal128 out; RETURN_NOT_OK(FromString(s, &out, nullptr, nullptr)); - return std::move(out); + return out; } Result Decimal128::FromString(const std::string& s) { @@ -850,7 +850,7 @@ Status Decimal256::FromString(const char* s, Decimal256* out, int32_t* precision Result Decimal256::FromString(std::string_view s) { Decimal256 out; RETURN_NOT_OK(FromString(s, &out, nullptr, nullptr)); - return std::move(out); + return out; } Result Decimal256::FromString(const std::string& s) { diff --git a/cpp/src/arrow/util/future.cc b/cpp/src/arrow/util/future.cc index a5426f949e721..60687172fe8d7 100644 --- a/cpp/src/arrow/util/future.cc +++ b/cpp/src/arrow/util/future.cc @@ -212,7 +212,7 @@ std::unique_ptr FutureImpl::Make() { std::unique_ptr FutureImpl::MakeFinished(FutureState state) { std::unique_ptr ptr(new ConcreteFutureImpl()); ptr->state_ = state; - return std::move(ptr); + return ptr; } FutureImpl::FutureImpl() : state_(FutureState::PENDING) {} diff --git a/cpp/src/arrow/util/future.h b/cpp/src/arrow/util/future.h index 283b581a5100a..0aa2842703712 100644 --- a/cpp/src/arrow/util/future.h +++ b/cpp/src/arrow/util/future.h @@ -871,7 +871,7 @@ Future ToFuture(Result maybe_value) { template Future ToFuture(Future fut) { - return std::move(fut); + return fut; } template diff --git a/cpp/src/arrow/util/io_util.cc b/cpp/src/arrow/util/io_util.cc index d48f9eb97d562..2eefe96f0d5c4 100644 --- a/cpp/src/arrow/util/io_util.cc +++ b/cpp/src/arrow/util/io_util.cc @@ -1084,7 +1084,7 @@ Result FileOpenReadable(const PlatformFilename& file_name) { } #endif - return std::move(fd); + return fd; } Result FileOpenWritable(const PlatformFilename& file_name, @@ -1148,7 +1148,7 @@ Result FileOpenWritable(const PlatformFilename& file_name, // Seek to end, as O_APPEND does not necessarily do it RETURN_NOT_OK(lseek64_compat(fd.fd(), 0, SEEK_END)); } - return std::move(fd); + return fd; } Result FileTell(int fd) { @@ -1967,7 +1967,7 @@ Result> TemporaryDir::Make(const std::string& pref for (const auto& base_dir : base_dirs) { ARROW_ASSIGN_OR_RAISE(auto ptr, TryCreatingDirectory(base_dir)); if (ptr) { - return std::move(ptr); + return ptr; } // Cannot create in this directory, try the next one } diff --git a/cpp/src/arrow/util/iterator.h b/cpp/src/arrow/util/iterator.h index 5e716d0fd113d..4da8394a0299c 100644 --- a/cpp/src/arrow/util/iterator.h +++ b/cpp/src/arrow/util/iterator.h @@ -180,9 +180,7 @@ class Iterator : public util::EqualityComparable> { ARROW_ASSIGN_OR_RAISE(auto element, maybe_element); out.push_back(std::move(element)); } - // ARROW-8193: On gcc-4.8 without the explicit move it tries to use the - // copy constructor, which may be deleted on the elements of type T - return std::move(out); + return out; } private: diff --git a/cpp/src/arrow/util/vector.h b/cpp/src/arrow/util/vector.h index e3c0a67cf46c4..74b6a2403a2bb 100644 --- a/cpp/src/arrow/util/vector.h +++ b/cpp/src/arrow/util/vector.h @@ -113,7 +113,7 @@ Result> MaybeMapVector(Fn&& map, const std::vector& source out.reserve(source.size()); ARROW_RETURN_NOT_OK(MaybeTransform(source.begin(), source.end(), std::back_inserter(out), std::forward(map))); - return std::move(out); + return out; } template , @@ -152,7 +152,7 @@ Result> UnwrapOrRaise(std::vector>&& results) { } out.push_back(it->MoveValueUnsafe()); } - return std::move(out); + return out; } template @@ -165,7 +165,7 @@ Result> UnwrapOrRaise(const std::vector>& results) { } out.push_back(result.ValueUnsafe()); } - return std::move(out); + return out; } } // namespace internal diff --git a/cpp/src/gandiva/function_registry.cc b/cpp/src/gandiva/function_registry.cc index 2e392630ee009..0955a2e47fcaf 100644 --- a/cpp/src/gandiva/function_registry.cc +++ b/cpp/src/gandiva/function_registry.cc @@ -147,7 +147,7 @@ arrow::Result> MakeDefaultFunctionRegistry() { ARROW_RETURN_NOT_OK(registry->Add(func_signature)); } } - return std::move(registry); + return registry; } std::shared_ptr default_function_registry() { diff --git a/cpp/src/gandiva/llvm_generator.cc b/cpp/src/gandiva/llvm_generator.cc index 62ebab08f4d6b..4afa2935ace33 100644 --- a/cpp/src/gandiva/llvm_generator.cc +++ b/cpp/src/gandiva/llvm_generator.cc @@ -1251,7 +1251,7 @@ LValuePtr LLVMGenerator::Visitor::BuildFunctionCall(const NativeFunction* func, // Make the function call auto out = decimalIR.CallDecimalFunction(func->pc_name(), llvm_return_type, *params); ret_lvalue->set_data(out); - return std::move(ret_lvalue); + return ret_lvalue; } else { bool isDecimalFunction = false; for (auto& arg : *params) { diff --git a/cpp/src/parquet/arrow/arrow_reader_writer_test.cc b/cpp/src/parquet/arrow/arrow_reader_writer_test.cc index dd0b19c2ce048..aad1e933c4f25 100644 --- a/cpp/src/parquet/arrow/arrow_reader_writer_test.cc +++ b/cpp/src/parquet/arrow/arrow_reader_writer_test.cc @@ -4469,7 +4469,7 @@ class TestArrowReadDictionary : public ::testing::TestWithParam { RETURN_NOT_OK(builder.Open(std::make_shared(buffer_))); RETURN_NOT_OK(builder.properties(properties_)->Build(&reader)); - return std::move(reader); + return reader; } }; diff --git a/cpp/src/parquet/encoding.cc b/cpp/src/parquet/encoding.cc index 004cb746b3a89..903faa92b6370 100644 --- a/cpp/src/parquet/encoding.cc +++ b/cpp/src/parquet/encoding.cc @@ -551,7 +551,7 @@ class DictEncoderImpl : public EncoderImpl, virtual public DictEncoder { int result_size = WriteIndices(buffer->mutable_data(), static_cast(EstimatedDataEncodedSize())); PARQUET_THROW_NOT_OK(buffer->Resize(result_size, false)); - return std::move(buffer); + return buffer; } /// Writes out the encoded dictionary to buffer. buffer must be preallocated to diff --git a/cpp/src/parquet/encryption/file_key_unwrapper.cc b/cpp/src/parquet/encryption/file_key_unwrapper.cc index 02bea127fd1c3..d88aa6c52ac12 100644 --- a/cpp/src/parquet/encryption/file_key_unwrapper.cc +++ b/cpp/src/parquet/encryption/file_key_unwrapper.cc @@ -124,7 +124,7 @@ KeyWithMasterId FileKeyUnwrapper::GetDataEncryptionKey(const KeyMaterial& key_ma data_key = internal::DecryptKeyLocally(encoded_wrapped_dek, kek_bytes, aad); } - return KeyWithMasterId(std::move(data_key), std::move(master_key_id)); + return KeyWithMasterId(std::move(data_key), master_key_id); } std::shared_ptr FileKeyUnwrapper::GetKmsClientFromConfigOrKeyMaterial( diff --git a/cpp/src/parquet/platform.cc b/cpp/src/parquet/platform.cc index 5c355c28be1c3..98946029fb866 100644 --- a/cpp/src/parquet/platform.cc +++ b/cpp/src/parquet/platform.cc @@ -35,7 +35,7 @@ std::shared_ptr<::arrow::io::BufferOutputStream> CreateOutputStream(MemoryPool* std::shared_ptr AllocateBuffer(MemoryPool* pool, int64_t size) { PARQUET_ASSIGN_OR_THROW(auto result, ::arrow::AllocateResizableBuffer(size, pool)); - return std::move(result); + return result; } } // namespace parquet diff --git a/cpp/src/parquet/properties.cc b/cpp/src/parquet/properties.cc index 2267efdf8a44a..4e6c558e064d4 100644 --- a/cpp/src/parquet/properties.cc +++ b/cpp/src/parquet/properties.cc @@ -38,7 +38,7 @@ std::shared_ptr ReaderProperties::GetStream( PARQUET_ASSIGN_OR_THROW( auto stream, ::arrow::io::BufferedInputStream::Create(buffer_size_, pool_, safe_stream, num_bytes)); - return std::move(stream); + return stream; } else { PARQUET_ASSIGN_OR_THROW(auto data, source->ReadAt(start, num_bytes)); diff --git a/cpp/src/skyhook/cls/cls_skyhook.cc b/cpp/src/skyhook/cls/cls_skyhook.cc index e021cb3c8248a..632b82f1d1a6c 100644 --- a/cpp/src/skyhook/cls/cls_skyhook.cc +++ b/cpp/src/skyhook/cls/cls_skyhook.cc @@ -95,7 +95,7 @@ class RandomAccessObject : public arrow::io::RandomAccessFile { arrow::Result> Read(int64_t nbytes) override { ARROW_ASSIGN_OR_RAISE(auto buffer, ReadAt(pos_, nbytes)); pos_ += buffer->size(); - return std::move(buffer); + return buffer; } /// Read a specified number of bytes from the current position into an output stream.