Skip to content

Commit

Permalink
apacheGH-41478: [C++] Clean up more redundant move warnings (apache#4…
Browse files Browse the repository at this point in the history
…1487)

### 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: apache#41478

Authored-by: Will Ayd <william.ayd@icloud.com>
Signed-off-by: Benjamin Kietzman <bengilgit@gmail.com>
  • Loading branch information
WillAyd authored May 24, 2024
1 parent b275483 commit 3a4fcff
Show file tree
Hide file tree
Showing 87 changed files with 218 additions and 198 deletions.
3 changes: 2 additions & 1 deletion cpp/cmake_modules/SetupCxxFlags.cmake
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
6 changes: 3 additions & 3 deletions cpp/src/arrow/acero/aggregate_internal.cc
Original file line number Diff line number Diff line change
Expand Up @@ -102,7 +102,7 @@ Result<std::unique_ptr<KernelState>> 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<std::vector<const HashAggregateKernel*>> GetKernels(
Expand All @@ -129,7 +129,7 @@ Result<std::vector<std::unique_ptr<KernelState>>> InitKernels(
ARROW_ASSIGN_OR_RAISE(states[i],
InitKernel(kernels[i], ctx, aggregates[i], in_types[i]));
}
return std::move(states);
return states;
}

Result<FieldVector> ResolveKernels(
Expand Down Expand Up @@ -242,7 +242,7 @@ Result<std::vector<Datum>> ExtractValues(const ExecBatch& input_batch,
DCHECK(false);
}
}
return std::move(values);
return values;
}

} // namespace aggregate
Expand Down
2 changes: 1 addition & 1 deletion cpp/src/arrow/acero/backpressure_handler.h
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down
2 changes: 1 addition & 1 deletion cpp/src/arrow/acero/hash_aggregate_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -318,7 +318,7 @@ Result<Datum> 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}},
})
Expand Down
2 changes: 1 addition & 1 deletion cpp/src/arrow/acero/hash_join.cc
Original file line number Diff line number Diff line change
Expand Up @@ -791,7 +791,7 @@ class HashJoinBasicImpl : public HashJoinImpl {

Result<std::unique_ptr<HashJoinImpl>> HashJoinImpl::MakeBasic() {
std::unique_ptr<HashJoinImpl> impl{new HashJoinBasicImpl()};
return std::move(impl);
return impl;
}

} // namespace acero
Expand Down
4 changes: 2 additions & 2 deletions cpp/src/arrow/acero/hash_join_node.cc
Original file line number Diff line number Diff line change
Expand Up @@ -351,7 +351,7 @@ Result<Expression> 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;
Expand Down Expand Up @@ -386,7 +386,7 @@ Result<Expression> HashJoinSchema::BindFilter(Expression filter,
filter.ToString(), " evaluates to ",
filter.type()->ToString());
}
return std::move(filter);
return filter;
}

Expression HashJoinSchema::RewriteFilterToUseFilterSchema(
Expand Down
4 changes: 2 additions & 2 deletions cpp/src/arrow/acero/order_by_impl.cc
Original file line number Diff line number Diff line change
Expand Up @@ -93,14 +93,14 @@ Result<std::unique_ptr<OrderByImpl>> OrderByImpl::MakeSort(
ExecContext* ctx, const std::shared_ptr<Schema>& output_schema,
const SortOptions& options) {
std::unique_ptr<OrderByImpl> impl{new SortBasicImpl(ctx, output_schema, options)};
return std::move(impl);
return impl;
}

Result<std::unique_ptr<OrderByImpl>> OrderByImpl::MakeSelectK(
ExecContext* ctx, const std::shared_ptr<Schema>& output_schema,
const SelectKOptions& options) {
std::unique_ptr<OrderByImpl> impl{new SelectKBasicImpl(ctx, output_schema, options)};
return std::move(impl);
return impl;
}

} // namespace acero
Expand Down
2 changes: 1 addition & 1 deletion cpp/src/arrow/acero/swiss_join.cc
Original file line number Diff line number Diff line change
Expand Up @@ -2985,7 +2985,7 @@ class SwissJoin : public HashJoinImpl {

Result<std::unique_ptr<HashJoinImpl>> HashJoinImpl::MakeSwiss() {
std::unique_ptr<HashJoinImpl> impl{new SwissJoin()};
return std::move(impl);
return impl;
}

} // namespace acero
Expand Down
2 changes: 1 addition & 1 deletion cpp/src/arrow/acero/task_util.cc
Original file line number Diff line number Diff line change
Expand Up @@ -424,7 +424,7 @@ void TaskSchedulerImpl::Abort(AbortContinuationImpl impl) {

std::unique_ptr<TaskScheduler> TaskScheduler::Make() {
std::unique_ptr<TaskSchedulerImpl> impl{new TaskSchedulerImpl()};
return std::move(impl);
return impl;
}

} // namespace acero
Expand Down
14 changes: 7 additions & 7 deletions cpp/src/arrow/acero/tpch_node.cc
Original file line number Diff line number Diff line change
Expand Up @@ -336,7 +336,7 @@ Result<Datum> 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,
Expand Down Expand Up @@ -611,7 +611,7 @@ Result<Datum> 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) {
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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;
}
}
{
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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<std::mutex> 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;
}

Expand Down
4 changes: 2 additions & 2 deletions cpp/src/arrow/adapters/orc/adapter.cc
Original file line number Diff line number Diff line change
Expand Up @@ -566,7 +566,7 @@ Result<std::unique_ptr<ORCFileReader>> ORCFileReader::Open(
#endif
auto result = std::unique_ptr<ORCFileReader>(new ORCFileReader());
RETURN_NOT_OK(result->impl_->Open(file, pool));
return std::move(result);
return result;
}

Result<std::shared_ptr<const KeyValueMetadata>> ORCFileReader::ReadMetadata() {
Expand Down Expand Up @@ -837,7 +837,7 @@ Result<std::unique_ptr<ORCFileWriter>> ORCFileWriter::Open(
std::unique_ptr<ORCFileWriter>(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); }
Expand Down
6 changes: 3 additions & 3 deletions cpp/src/arrow/adapters/orc/util.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1026,7 +1026,7 @@ Result<std::unique_ptr<liborc::Type>> 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<const MapType&>(type).key_field();
Expand All @@ -1048,7 +1048,7 @@ Result<std::unique_ptr<liborc::Type>> 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: ",
Expand Down Expand Up @@ -1195,7 +1195,7 @@ Result<std::unique_ptr<liborc::Type>> 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<std::shared_ptr<const KeyValueMetadata>> GetFieldMetadata(
Expand Down
2 changes: 1 addition & 1 deletion cpp/src/arrow/array/array_nested.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1177,7 +1177,7 @@ void SparseUnionArray::SetData(std::shared_ptr<ArrayData> data) {
}

void DenseUnionArray::SetData(const std::shared_ptr<ArrayData>& 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);
Expand Down
6 changes: 3 additions & 3 deletions cpp/src/arrow/array/builder_base.h
Original file line number Diff line number Diff line change
Expand Up @@ -332,7 +332,7 @@ inline Result<std::unique_ptr<ArrayBuilder>> MakeBuilder(
const std::shared_ptr<DataType>& type, MemoryPool* pool = default_memory_pool()) {
std::unique_ptr<ArrayBuilder> out;
ARROW_RETURN_NOT_OK(MakeBuilder(pool, type, &out));
return std::move(out);
return out;
}

/// \brief Construct an empty ArrayBuilder corresponding to the data
Expand All @@ -346,7 +346,7 @@ inline Result<std::unique_ptr<ArrayBuilder>> MakeBuilderExactIndex(
const std::shared_ptr<DataType>& type, MemoryPool* pool = default_memory_pool()) {
std::unique_ptr<ArrayBuilder> out;
ARROW_RETURN_NOT_OK(MakeBuilderExactIndex(pool, type, &out));
return std::move(out);
return out;
}

/// \brief Construct an empty DictionaryBuilder initialized optionally
Expand All @@ -365,7 +365,7 @@ inline Result<std::unique_ptr<ArrayBuilder>> MakeDictionaryBuilder(
MemoryPool* pool = default_memory_pool()) {
std::unique_ptr<ArrayBuilder> out;
ARROW_RETURN_NOT_OK(MakeDictionaryBuilder(pool, type, dictionary, &out));
return std::move(out);
return out;
}

} // namespace arrow
3 changes: 1 addition & 2 deletions cpp/src/arrow/array/builder_run_end.cc
Original file line number Diff line number Diff line change
Expand Up @@ -162,8 +162,7 @@ Status RunCompressorBuilder::FinishInternal(std::shared_ptr<ArrayData>* out) {
RunEndEncodedBuilder::ValueRunBuilder::ValueRunBuilder(
MemoryPool* pool, const std::shared_ptr<ArrayBuilder>& value_builder,
const std::shared_ptr<DataType>& 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<ArrayBuilder>& run_end_builder,
Expand Down
3 changes: 2 additions & 1 deletion cpp/src/arrow/array/concatenate.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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<Buffer>(std::move(out));
}

Status Visit(const DictionaryType& d) {
Expand Down
3 changes: 2 additions & 1 deletion cpp/src/arrow/array/util.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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<Buffer>(std::move(out_buffer));
}

template <typename VALUE_TYPE>
Expand Down
12 changes: 8 additions & 4 deletions cpp/src/arrow/buffer.cc
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,8 @@ Result<std::shared_ptr<Buffer>> 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<size_t>(nbytes));
return std::move(new_buffer);
// R build with openSUSE155 requires an explicit shared_ptr construction
return std::shared_ptr<Buffer>(std::move(new_buffer));
}

Buffer::Buffer() : Buffer(memory_pool::internal::kZeroSizeArea, 0) {}
Expand Down Expand Up @@ -185,7 +186,8 @@ Result<std::shared_ptr<Buffer>> 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<Buffer>(std::move(buf));
}

Result<std::shared_ptr<Buffer>> AllocateEmptyBitmap(int64_t length, MemoryPool* pool) {
Expand All @@ -197,7 +199,8 @@ Result<std::shared_ptr<Buffer>> 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<size_t>(buf->size()));
return std::move(buf);
// R build with openSUSE155 requires an explicit shared_ptr construction
return std::shared_ptr<Buffer>(std::move(buf));
}

Status AllocateEmptyBitmap(int64_t length, std::shared_ptr<Buffer>* out) {
Expand All @@ -219,7 +222,8 @@ Result<std::shared_ptr<Buffer>> ConcatenateBuffers(
out_data += buffer->size();
}
}
return std::move(out);
// R build with openSUSE155 requires an explicit shared_ptr construction
return std::shared_ptr<Buffer>(std::move(out));
}

} // namespace arrow
2 changes: 1 addition & 1 deletion cpp/src/arrow/c/bridge_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1362,7 +1362,7 @@ class MyMemoryManager : public CPUMemoryManager {
if (buf.size() > 0) {
memcpy(dest->mutable_data(), buf.data(), static_cast<size_t>(buf.size()));
}
return std::move(dest);
return dest;
}
};

Expand Down
8 changes: 4 additions & 4 deletions cpp/src/arrow/compute/exec.cc
Original file line number Diff line number Diff line change
Expand Up @@ -923,7 +923,7 @@ class ScalarExecutor : public KernelExecutorImpl<ScalarKernel> {
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();
}
Expand Down Expand Up @@ -1107,7 +1107,7 @@ class VectorExecutor : public KernelExecutorImpl<VectorKernel> {
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) {
Expand All @@ -1116,10 +1116,10 @@ class VectorExecutor : public KernelExecutorImpl<VectorKernel> {
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);
}
}

Expand Down
4 changes: 2 additions & 2 deletions cpp/src/arrow/compute/expression.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1645,7 +1645,7 @@ Expression and_(const std::vector<Expression>& 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;
}
Expand All @@ -1659,7 +1659,7 @@ Expression or_(const std::vector<Expression>& 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;
}
Expand Down
Loading

0 comments on commit 3a4fcff

Please sign in to comment.