diff --git a/cpp/src/arrow/array/array_base.h b/cpp/src/arrow/array/array_base.h index 48d65e30de8d9..7e857bf20568e 100644 --- a/cpp/src/arrow/array/array_base.h +++ b/cpp/src/arrow/array/array_base.h @@ -106,7 +106,7 @@ class ARROW_EXPORT Array { /// \see GetNullCount int64_t ComputeLogicalNullCount() const; - std::shared_ptr type() const { return data_->type; } + const std::shared_ptr& type() const { return data_->type; } Type::type type_id() const { return data_->type->id(); } /// Buffer for the validity (null) bitmap, if any. Note that Union types @@ -251,7 +251,7 @@ class ARROW_EXPORT PrimitiveArray : public FlatArray { int64_t null_count = kUnknownNullCount, int64_t offset = 0); /// Does not account for any slice offset - std::shared_ptr values() const { return data_->buffers[1]; } + const std::shared_ptr& values() const { return data_->buffers[1]; } protected: PrimitiveArray() : raw_values_(NULLPTR) {} diff --git a/cpp/src/arrow/array/array_dict.cc b/cpp/src/arrow/array/array_dict.cc index 8fbe9f69d7897..cccc7bb78220d 100644 --- a/cpp/src/arrow/array/array_dict.cc +++ b/cpp/src/arrow/array/array_dict.cc @@ -50,7 +50,7 @@ using internal::CopyBitmap; // ---------------------------------------------------------------------- // DictionaryArray -std::shared_ptr DictionaryArray::indices() const { return indices_; } +const std::shared_ptr& DictionaryArray::indices() const { return indices_; } int64_t DictionaryArray::GetValueIndex(int64_t i) const { const uint8_t* indices_data = data_->buffers[1]->data(); @@ -106,8 +106,9 @@ DictionaryArray::DictionaryArray(const std::shared_ptr& type, SetData(data); } -std::shared_ptr DictionaryArray::dictionary() const { +const std::shared_ptr& DictionaryArray::dictionary() const { if (!dictionary_) { + // TODO(GH-36503) this isn't thread safe dictionary_ = MakeArray(data_->dictionary); } return dictionary_; diff --git a/cpp/src/arrow/array/array_dict.h b/cpp/src/arrow/array/array_dict.h index 8791eaa07db3a..b7d4db4b415b4 100644 --- a/cpp/src/arrow/array/array_dict.h +++ b/cpp/src/arrow/array/array_dict.h @@ -101,8 +101,8 @@ class ARROW_EXPORT DictionaryArray : public Array { /// \brief Return the dictionary for this array, which is stored as /// a member of the ArrayData internal structure - std::shared_ptr dictionary() const; - std::shared_ptr indices() const; + const std::shared_ptr& dictionary() const; + const std::shared_ptr& indices() const; /// \brief Return the ith value of indices, cast to int64_t. Not recommended /// for use in performance-sensitive code. Does not validate whether the diff --git a/cpp/src/arrow/array/array_nested.cc b/cpp/src/arrow/array/array_nested.cc index f99163206ea57..61eeb496e5a5b 100644 --- a/cpp/src/arrow/array/array_nested.cc +++ b/cpp/src/arrow/array/array_nested.cc @@ -470,11 +470,11 @@ const FixedSizeListType* FixedSizeListArray::list_type() const { return checked_cast(data_->type.get()); } -std::shared_ptr FixedSizeListArray::value_type() const { +const std::shared_ptr& FixedSizeListArray::value_type() const { return list_type()->value_type(); } -std::shared_ptr FixedSizeListArray::values() const { return values_; } +const std::shared_ptr& FixedSizeListArray::values() const { return values_; } Result> FixedSizeListArray::FromArrays( const std::shared_ptr& values, int32_t list_size) { diff --git a/cpp/src/arrow/array/array_nested.h b/cpp/src/arrow/array/array_nested.h index 4f5f3f614cb64..47c1db039ccc9 100644 --- a/cpp/src/arrow/array/array_nested.h +++ b/cpp/src/arrow/array/array_nested.h @@ -71,12 +71,12 @@ class BaseListArray : public Array { /// \brief Return array object containing the list's values /// /// Note that this buffer does not account for any slice offset or length. - std::shared_ptr values() const { return values_; } + const std::shared_ptr& values() const { return values_; } /// Note that this buffer does not account for any slice offset or length. - std::shared_ptr value_offsets() const { return data_->buffers[1]; } + const std::shared_ptr& value_offsets() const { return data_->buffers[1]; } - std::shared_ptr value_type() const { return list_type_->value_type(); } + const std::shared_ptr& value_type() const { return list_type_->value_type(); } /// Return pointer to raw value offsets accounting for any slice offset const offset_type* raw_value_offsets() const { @@ -269,10 +269,10 @@ class ARROW_EXPORT MapArray : public ListArray { const MapType* map_type() const { return map_type_; } /// \brief Return array object containing all map keys - std::shared_ptr keys() const { return keys_; } + const std::shared_ptr& keys() const { return keys_; } /// \brief Return array object containing all mapped items - std::shared_ptr items() const { return items_; } + const std::shared_ptr& items() const { return items_; } /// Validate child data before constructing the actual MapArray. static Status ValidateChildData( @@ -310,9 +310,9 @@ class ARROW_EXPORT FixedSizeListArray : public Array { const FixedSizeListType* list_type() const; /// \brief Return array object containing the list's values - std::shared_ptr values() const; + const std::shared_ptr& values() const; - std::shared_ptr value_type() const; + const std::shared_ptr& value_type() const; // The following functions will not perform boundschecking int64_t value_offset(int64_t i) const { @@ -432,7 +432,7 @@ class ARROW_EXPORT UnionArray : public Array { using type_code_t = int8_t; /// Note that this buffer does not account for any slice offset - std::shared_ptr type_codes() const { return data_->buffers[1]; } + const std::shared_ptr& type_codes() const { return data_->buffers[1]; } const type_code_t* raw_type_codes() const { return raw_type_codes_ + data_->offset; } @@ -571,7 +571,7 @@ class ARROW_EXPORT DenseUnionArray : public UnionArray { } /// Note that this buffer does not account for any slice offset - std::shared_ptr value_offsets() const { return data_->buffers[2]; } + const std::shared_ptr& value_offsets() const { return data_->buffers[2]; } int32_t value_offset(int64_t i) const { return raw_value_offsets_[i + data_->offset]; } diff --git a/cpp/src/arrow/array/data.h b/cpp/src/arrow/array/data.h index 715c233fcf2b4..8c6b250b71adf 100644 --- a/cpp/src/arrow/array/data.h +++ b/cpp/src/arrow/array/data.h @@ -360,6 +360,15 @@ struct ARROW_EXPORT BufferSpan { int64_t size = 0; // Pointer back to buffer that owns this memory const std::shared_ptr* owner = NULLPTR; + + template + const T* data_as() const { + return reinterpret_cast(data); + } + template + T* mutable_data_as() { + return reinterpret_cast(data); + } }; /// \brief EXPERIMENTAL: A non-owning ArrayData reference that is cheaply diff --git a/cpp/src/arrow/buffer.cc b/cpp/src/arrow/buffer.cc index afe3d773594f0..135aa0c5328ad 100644 --- a/cpp/src/arrow/buffer.cc +++ b/cpp/src/arrow/buffer.cc @@ -21,6 +21,7 @@ #include #include +#include "arrow/memory_pool_internal.h" #include "arrow/result.h" #include "arrow/status.h" #include "arrow/util/bit_util.h" @@ -43,6 +44,8 @@ Result> Buffer::CopySlice(const int64_t start, return std::move(new_buffer); } +Buffer::Buffer() : Buffer(memory_pool::internal::kZeroSizeArea, 0) {} + namespace { Status CheckBufferSlice(const Buffer& buffer, int64_t offset, int64_t length) { @@ -147,11 +150,12 @@ Result> Buffer::ViewOrCopy( class StlStringBuffer : public Buffer { public: - explicit StlStringBuffer(std::string data) - : Buffer(nullptr, 0), input_(std::move(data)) { - data_ = reinterpret_cast(input_.c_str()); - size_ = static_cast(input_.size()); - capacity_ = size_; + explicit StlStringBuffer(std::string data) : input_(std::move(data)) { + if (!input_.empty()) { + data_ = reinterpret_cast(input_.c_str()); + size_ = static_cast(input_.size()); + capacity_ = size_; + } } private: diff --git a/cpp/src/arrow/buffer.h b/cpp/src/arrow/buffer.h index 238768b815b62..ad2496aeeb5a6 100644 --- a/cpp/src/arrow/buffer.h +++ b/cpp/src/arrow/buffer.h @@ -49,6 +49,8 @@ namespace arrow { /// The following invariant is always true: Size <= Capacity class ARROW_EXPORT Buffer { public: + ARROW_DISALLOW_COPY_AND_ASSIGN(Buffer); + /// \brief Construct from buffer and size without copying memory /// /// \param[in] data a memory buffer @@ -136,6 +138,32 @@ class ARROW_EXPORT Buffer { /// \return a new Buffer instance static std::shared_ptr FromString(std::string data); + /// \brief Construct an immutable buffer that takes ownership of the contents + /// of an std::vector (without copying it). Only vectors of TrivialType objects + /// (integers, floating point numbers, ...) can be wrapped by this function. + /// + /// \param[in] vec a vector to own + /// \return a new Buffer instance + template + static std::shared_ptr FromVector(std::vector vec) { + static_assert(std::is_trivial_v, + "Buffer::FromVector can only wrap vectors of trivial objects"); + + if (vec.empty()) { + return std::shared_ptr{new Buffer()}; + } + + auto* data = reinterpret_cast(vec.data()); + auto size_in_bytes = static_cast(vec.size() * sizeof(T)); + return std::shared_ptr{ + new Buffer{data, size_in_bytes}, + // Keep the vector's buffer alive inside the shared_ptr's destructor until after + // we have deleted the Buffer. Note we can't use this trick in FromString since + // std::string's data is inline for short strings so moving invalidates pointers + // into the string's buffer. + [vec = std::move(vec)](Buffer* buffer) { delete buffer; }}; + } + /// \brief Create buffer referencing typed memory with some length without /// copying /// \param[in] data the typed memory as C array @@ -182,6 +210,15 @@ class ARROW_EXPORT Buffer { return ARROW_PREDICT_TRUE(is_cpu_) ? data_ : NULLPTR; } + /// \brief Return a pointer to the buffer's data cast to a specific type + /// + /// The buffer has to be a CPU buffer (`is_cpu()` is true). + /// Otherwise, an assertion may be thrown or a null pointer may be returned. + template + const T* data_as() const { + return reinterpret_cast(data()); + } + /// \brief Return a writable pointer to the buffer's data /// /// The buffer has to be a mutable CPU buffer (`is_cpu()` and `is_mutable()` @@ -199,6 +236,16 @@ class ARROW_EXPORT Buffer { : NULLPTR; } + /// \brief Return a writable pointer to the buffer's data cast to a specific type + /// + /// The buffer has to be a mutable CPU buffer (`is_cpu()` and `is_mutable()` + /// are true). Otherwise, an assertion may be thrown or a null pointer may + /// be returned. + template + T* mutable_data_as() { + return reinterpret_cast(mutable_data()); + } + /// \brief Return the device address of the buffer's data uintptr_t address() const { return reinterpret_cast(data_); } @@ -298,6 +345,8 @@ class ARROW_EXPORT Buffer { std::shared_ptr memory_manager_; protected: + Buffer(); + void CheckMutable() const; void CheckCPU() const; @@ -305,10 +354,6 @@ class ARROW_EXPORT Buffer { memory_manager_ = std::move(mm); is_cpu_ = memory_manager_->is_cpu(); } - - private: - Buffer() = delete; - ARROW_DISALLOW_COPY_AND_ASSIGN(Buffer); }; /// \defgroup buffer-slicing-functions Functions for slicing buffers diff --git a/cpp/src/arrow/buffer_builder.h b/cpp/src/arrow/buffer_builder.h index 5f37e552004a1..e7eea64043ba8 100644 --- a/cpp/src/arrow/buffer_builder.h +++ b/cpp/src/arrow/buffer_builder.h @@ -118,6 +118,11 @@ class ARROW_EXPORT BufferBuilder { return Status::OK(); } + /// \brief Append the given data to the buffer + /// + /// The buffer is automatically expanded if necessary. + Status Append(std::string_view v) { return Append(v.data(), v.size()); } + /// \brief Append copies of a value to the buffer /// /// The buffer is automatically expanded if necessary. @@ -138,6 +143,7 @@ class ARROW_EXPORT BufferBuilder { memcpy(data_ + size_, data, static_cast(length)); size_ += length; } + void UnsafeAppend(std::string_view v) { UnsafeAppend(v.data(), v.size()); } void UnsafeAppend(const int64_t num_copies, uint8_t value) { memset(data_ + size_, value, static_cast(num_copies)); @@ -196,6 +202,14 @@ class ARROW_EXPORT BufferBuilder { int64_t length() const { return size_; } const uint8_t* data() const { return data_; } uint8_t* mutable_data() { return data_; } + template + const T* data_as() const { + return reinterpret_cast(data_); + } + template + T* mutable_data_as() { + return reinterpret_cast(data_); + } private: std::shared_ptr buffer_; diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 677b19138995e..47cae538e2e3f 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -246,10 +246,10 @@ struct GroupedCountAllImpl : public GroupedAggregator { const ArrayData& group_id_mapping) override { auto other = checked_cast(&raw_other); - auto counts = reinterpret_cast(counts_.mutable_data()); - auto other_counts = reinterpret_cast(other->counts_.data()); + auto* counts = counts_.mutable_data_as(); + const auto* other_counts = other->counts_.data_as(); - auto g = group_id_mapping.GetValues(1); + auto* g = group_id_mapping.GetValues(1); for (int64_t other_g = 0; other_g < group_id_mapping.length; ++other_g, ++g) { counts[*g] += other_counts[other_g]; } @@ -257,8 +257,8 @@ struct GroupedCountAllImpl : public GroupedAggregator { } Status Consume(const ExecSpan& batch) override { - auto counts = reinterpret_cast(counts_.mutable_data()); - auto g_begin = batch[0].array.GetValues(1); + auto* counts = counts_.mutable_data_as(); + auto* g_begin = batch[0].array.GetValues(1); for (auto g_itr = g_begin, end = g_itr + batch.length; g_itr != end; g_itr++) { counts[*g_itr] += 1; } @@ -293,10 +293,10 @@ struct GroupedCountImpl : public GroupedAggregator { const ArrayData& group_id_mapping) override { auto other = checked_cast(&raw_other); - auto counts = reinterpret_cast(counts_.mutable_data()); - auto other_counts = reinterpret_cast(other->counts_.mutable_data()); + auto* counts = counts_.mutable_data_as(); + const auto* other_counts = other->counts_.data_as(); - auto g = group_id_mapping.GetValues(1); + auto* g = group_id_mapping.GetValues(1); for (int64_t other_g = 0; other_g < group_id_mapping.length; ++other_g, ++g) { counts[*g] += other_counts[other_g]; } @@ -344,8 +344,8 @@ struct GroupedCountImpl : public GroupedAggregator { }; Status Consume(const ExecSpan& batch) override { - auto counts = reinterpret_cast(counts_.mutable_data()); - auto g_begin = batch[1].array.GetValues(1); + auto* counts = counts_.mutable_data_as(); + auto* g_begin = batch[1].array.GetValues(1); if (options_.mode == CountOptions::ALL) { for (int64_t i = 0; i < batch.length; ++i, ++g_begin) { @@ -682,7 +682,7 @@ struct GroupedSumNullImpl final : public GroupedNullImpl { std::shared_ptr out_type() const override { return int64(); } void output_empty(const std::shared_ptr& data) override { - std::fill_n(reinterpret_cast(data->mutable_data()), num_groups_, 0); + std::fill_n(data->mutable_data_as(), num_groups_, 0); } }; @@ -722,7 +722,7 @@ struct GroupedProductNullImpl final : public GroupedNullImpl { std::shared_ptr out_type() const override { return int64(); } void output_empty(const std::shared_ptr& data) override { - std::fill_n(reinterpret_cast(data->mutable_data()), num_groups_, 1); + std::fill_n(data->mutable_data_as(), num_groups_, 1); } }; @@ -785,7 +785,7 @@ struct GroupedMeanImpl : public GroupedReducingAggregatordata(); ARROW_ASSIGN_OR_RAISE(std::shared_ptr values, AllocateBuffer(num_groups * sizeof(MeanType), pool)); - MeanType* means = reinterpret_cast(values->mutable_data()); + auto* means = values->mutable_data_as(); for (int64_t i = 0; i < num_groups; ++i) { if (counts[i] >= options.min_count) { ARROW_ASSIGN_OR_RAISE(means[i], DoMean(reduced[i], counts[i])); @@ -814,7 +814,7 @@ struct GroupedMeanNullImpl final : public GroupedNullImpl { std::shared_ptr out_type() const override { return float64(); } void output_empty(const std::shared_ptr& data) override { - std::fill_n(reinterpret_cast(data->mutable_data()), num_groups_, 0); + std::fill_n(data->mutable_data_as(), num_groups_, 0); } }; @@ -915,7 +915,7 @@ struct GroupedVarStdImpl : public GroupedAggregator { ARROW_ASSIGN_OR_RAISE(auto mapping, AllocateBuffer(num_groups_ * sizeof(uint32_t), pool_)); for (uint32_t i = 0; static_cast(i) < num_groups_; i++) { - reinterpret_cast(mapping->mutable_data())[i] = i; + mapping->template mutable_data_as()[i] = i; } ArrayData group_id_mapping(uint32(), num_groups_, {nullptr, std::move(mapping)}, /*null_count=*/0); @@ -932,7 +932,7 @@ struct GroupedVarStdImpl : public GroupedAggregator { // for int32: -2^62 <= sum < 2^62 constexpr int64_t max_length = 1ULL << (63 - sizeof(CType) * 8); - const auto g = batch[1].array.GetValues(1); + const auto* g = batch[1].array.GetValues(1); if (batch[0].is_scalar() && !batch[0].scalar->is_valid) { uint8_t* no_nulls = no_nulls_.mutable_data(); for (int64_t i = 0; i < batch.length; i++) { @@ -946,7 +946,7 @@ struct GroupedVarStdImpl : public GroupedAggregator { ARROW_ASSIGN_OR_RAISE(auto mapping, AllocateBuffer(num_groups_ * sizeof(uint32_t), pool_)); for (uint32_t i = 0; static_cast(i) < num_groups_; i++) { - reinterpret_cast(mapping->mutable_data())[i] = i; + mapping->template mutable_data_as()[i] = i; } ArrayData group_id_mapping(uint32(), num_groups_, {nullptr, std::move(mapping)}, /*null_count=*/0); @@ -1049,7 +1049,7 @@ struct GroupedVarStdImpl : public GroupedAggregator { AllocateBuffer(num_groups_ * sizeof(double), pool_)); int64_t null_count = 0; - double* results = reinterpret_cast(values->mutable_data()); + auto* results = values->mutable_data_as(); const int64_t* counts = counts_.data(); const double* m2s = m2s_.data(); for (int64_t i = 0; i < num_groups_; ++i) { @@ -1223,7 +1223,7 @@ struct GroupedTDigestImpl : public GroupedAggregator { AllocateBuffer(num_values * sizeof(double), pool_)); int64_t null_count = 0; - double* results = reinterpret_cast(values->mutable_data()); + auto* results = values->mutable_data_as(); for (int64_t i = 0; static_cast(i) < tdigests_.size(); ++i) { if (!tdigests_[i].is_empty() && counts[i] >= options_.min_count && (options_.skip_nulls || bit_util::GetBit(no_nulls_.data(), i))) { @@ -1567,7 +1567,7 @@ struct GroupedMinMaxImplmemory_pool())); - offset_type* offsets = reinterpret_cast(raw_offsets->mutable_data()); + auto* offsets = raw_offsets->mutable_data_as(); offsets[0] = 0; offsets++; const uint8_t* null_bitmap = array->buffers[0]->data(); @@ -2100,7 +2100,7 @@ struct GroupedFirstLastImplmemory_pool())); - offset_type* offsets = reinterpret_cast(raw_offsets->mutable_data()); + auto* offsets = raw_offsets->mutable_data_as(); offsets[0] = 0; offsets++; const uint8_t* null_bitmap = array->buffers[0]->data(); @@ -2464,9 +2464,9 @@ struct GroupedCountDistinctImpl : public GroupedAggregator { ARROW_ASSIGN_OR_RAISE(std::shared_ptr remapped_g, AllocateBuffer(uniques.length * sizeof(uint32_t), pool_)); - const auto* g_mapping = group_id_mapping.GetValues(1); - const auto* other_g = uniques[1].array()->GetValues(1); - auto* g = reinterpret_cast(remapped_g->mutable_data()); + const auto* g_mapping = group_id_mapping.buffers[1]->data_as(); + const auto* other_g = uniques[1].array()->buffers[1]->data_as(); + auto* g = remapped_g->mutable_data_as(); for (int64_t i = 0; i < uniques.length; i++) { g[i] = g_mapping[other_g[i]]; @@ -2480,7 +2480,7 @@ struct GroupedCountDistinctImpl : public GroupedAggregator { Result Finalize() override { ARROW_ASSIGN_OR_RAISE(std::shared_ptr values, AllocateBuffer(num_groups_ * sizeof(int64_t), pool_)); - int64_t* counts = reinterpret_cast(values->mutable_data()); + auto* counts = values->mutable_data_as(); std::fill(counts, counts + num_groups_, 0); ARROW_ASSIGN_OR_RAISE(auto uniques, grouper_->GetUniques()); @@ -2524,9 +2524,9 @@ struct GroupedDistinctImpl : public GroupedCountDistinctImpl { static_cast(num_groups_), ctx_)); ARROW_ASSIGN_OR_RAISE( auto list, grouper_->ApplyGroupings(*groupings, *uniques[0].make_array(), ctx_)); - auto values = list->values(); + const auto& values = list->values(); DCHECK_EQ(values->offset(), 0); - int32_t* offsets = reinterpret_cast(list->value_offsets()->mutable_data()); + auto* offsets = list->value_offsets()->mutable_data_as(); if (options_.mode == CountOptions::ALL || (options_.mode == CountOptions::ONLY_VALID && values->null_count() == 0)) { return list; @@ -2754,7 +2754,7 @@ struct GroupedOneImpl::value || ARROW_ASSIGN_OR_RAISE( auto raw_offsets, AllocateBuffer((1 + values.size()) * sizeof(offset_type), ctx_->memory_pool())); - auto* offsets = reinterpret_cast(raw_offsets->mutable_data()); + auto* offsets = raw_offsets->mutable_data_as(); offsets[0] = 0; offsets++; const uint8_t* null_bitmap = array->buffers[0]->data(); @@ -2952,7 +2952,7 @@ struct GroupedListImpl final : public GroupedAggregator { RETURN_NOT_OK(groups_.Append(g[other_raw_groups[other_g]])); } - const uint8_t* values = reinterpret_cast(other->values_.data()); + const auto* values = reinterpret_cast(other->values_.data()); RETURN_NOT_OK(GetSet::AppendBuffers(&values_, values, 0, other->num_args_)); if (other->has_nulls_) { @@ -3093,7 +3093,7 @@ struct GroupedListImpl::value || ARROW_ASSIGN_OR_RAISE( auto raw_offsets, AllocateBuffer((1 + values.size()) * sizeof(offset_type), ctx_->memory_pool())); - auto* offsets = reinterpret_cast(raw_offsets->mutable_data()); + auto* offsets = raw_offsets->mutable_data_as(); offsets[0] = 0; offsets++; const uint8_t* null_bitmap = array->buffers[0]->data(); diff --git a/cpp/src/arrow/type.h b/cpp/src/arrow/type.h index d218789f681fc..ccd1ddccf54ce 100644 --- a/cpp/src/arrow/type.h +++ b/cpp/src/arrow/type.h @@ -922,7 +922,7 @@ class ARROW_EXPORT BaseListType : public NestedType { ~BaseListType() override; const std::shared_ptr& value_field() const { return children_[0]; } - std::shared_ptr value_type() const { return children_[0]->type(); } + const std::shared_ptr& value_type() const { return children_[0]->type(); } }; /// \brief Concrete type class for list data diff --git a/cpp/src/parquet/column_reader.cc b/cpp/src/parquet/column_reader.cc index 3294aaaf283f1..6fe1ce9da60fe 100644 --- a/cpp/src/parquet/column_reader.cc +++ b/cpp/src/parquet/column_reader.cc @@ -43,6 +43,7 @@ #include "arrow/util/int_util_overflow.h" #include "arrow/util/logging.h" #include "arrow/util/rle_encoding.h" +#include "arrow/util/unreachable.h" #include "parquet/column_page.h" #include "parquet/encoding.h" #include "parquet/encryption/encryption_internal.h" @@ -103,7 +104,7 @@ inline void CheckNumberDecoded(int64_t number_decoded, int64_t expected) { LevelDecoder::LevelDecoder() : num_values_remaining_(0) {} -LevelDecoder::~LevelDecoder() {} +LevelDecoder::~LevelDecoder() = default; int LevelDecoder::SetData(Encoding::type encoding, int16_t max_level, int num_buffered_values, const uint8_t* data, @@ -435,9 +436,7 @@ std::shared_ptr SerializedPageReader::NextPage() { // until a maximum allowed header limit while (true) { PARQUET_ASSIGN_OR_THROW(auto view, stream_->Peek(allowed_page_size)); - if (view.size() == 0) { - return std::shared_ptr(nullptr); - } + if (view.size() == 0) return nullptr; // This gets used, then set by DeserializeThriftMsg header_size = static_cast(view.size()); @@ -1267,11 +1266,10 @@ int64_t TypedColumnReaderImpl::Skip(int64_t num_values_to_skip) { ARROW_DCHECK_NE(this->scratch_for_skip_, nullptr); do { int64_t batch_size = std::min(kSkipScratchBatchSize, values_to_skip); - values_read = ReadBatch( - static_cast(batch_size), - reinterpret_cast(this->scratch_for_skip_->mutable_data()), - reinterpret_cast(this->scratch_for_skip_->mutable_data()), - reinterpret_cast(this->scratch_for_skip_->mutable_data()), &values_read); + values_read = ReadBatch(static_cast(batch_size), + scratch_for_skip_->mutable_data_as(), + scratch_for_skip_->mutable_data_as(), + scratch_for_skip_->mutable_data_as(), &values_read); values_to_skip -= values_read; } while (values_read > 0 && values_to_skip > 0); } @@ -1315,8 +1313,7 @@ std::shared_ptr ColumnReader::Make(const ColumnDescriptor* descr, default: ParquetException::NYI("type reader not implemented"); } - // Unreachable code, but suppress compiler warning - return std::shared_ptr(nullptr); + ::arrow::Unreachable(); } // ---------------------------------------------------------------------- @@ -1454,7 +1451,7 @@ class TypedRecordReader : public TypedColumnReaderImpl, int64_t levels_remaining = levels_written_ - gap; auto left_shift = [&](::arrow::ResizableBuffer* buffer) { - int16_t* data = reinterpret_cast(buffer->mutable_data()); + auto* data = buffer->mutable_data_as(); std::copy(data + levels_position_, data + levels_written_, data + start_levels_position); PARQUET_THROW_NOT_OK(buffer->Resize(levels_remaining * sizeof(int16_t), @@ -1619,7 +1616,7 @@ class TypedRecordReader : public TypedColumnReaderImpl, do { int64_t batch_size = std::min(kSkipScratchBatchSize, values_left); values_read = this->ReadValues( - batch_size, reinterpret_cast(this->scratch_for_skip_->mutable_data())); + batch_size, this->scratch_for_skip_->template mutable_data_as()); values_left -= values_read; } while (values_read > 0 && values_left > 0); if (values_left > 0) { @@ -2033,7 +2030,7 @@ class TypedRecordReader : public TypedColumnReaderImpl, protected: template T* ValuesHead() { - return reinterpret_cast(values_->mutable_data()) + values_written_; + return values_->mutable_data_as() + values_written_; } LevelInfo leaf_info_; };