Back to home page

EIC code displayed by LXR

 
 

    


File indexing completed on 2025-08-28 08:26:54

0001 // Licensed to the Apache Software Foundation (ASF) under one
0002 // or more contributor license agreements.  See the NOTICE file
0003 // distributed with this work for additional information
0004 // regarding copyright ownership.  The ASF licenses this file
0005 // to you under the Apache License, Version 2.0 (the
0006 // "License"); you may not use this file except in compliance
0007 // with the License.  You may obtain a copy of the License at
0008 //
0009 //   http://www.apache.org/licenses/LICENSE-2.0
0010 //
0011 // Unless required by applicable law or agreed to in writing,
0012 // software distributed under the License is distributed on an
0013 // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
0014 // KIND, either express or implied.  See the License for the
0015 // specific language governing permissions and limitations
0016 // under the License.
0017 
0018 #pragma once
0019 
0020 #include <algorithm>
0021 #include <cstdint>
0022 #include <memory>
0023 #include <type_traits>
0024 
0025 #include "arrow/array/array_base.h"
0026 #include "arrow/array/array_binary.h"
0027 #include "arrow/array/builder_adaptive.h"   // IWYU pragma: export
0028 #include "arrow/array/builder_base.h"       // IWYU pragma: export
0029 #include "arrow/array/builder_primitive.h"  // IWYU pragma: export
0030 #include "arrow/array/data.h"
0031 #include "arrow/array/util.h"
0032 #include "arrow/scalar.h"
0033 #include "arrow/status.h"
0034 #include "arrow/type.h"
0035 #include "arrow/type_traits.h"
0036 #include "arrow/util/bit_block_counter.h"
0037 #include "arrow/util/checked_cast.h"
0038 #include "arrow/util/decimal.h"
0039 #include "arrow/util/macros.h"
0040 #include "arrow/util/visibility.h"
0041 
0042 namespace arrow {
0043 
0044 // ----------------------------------------------------------------------
0045 // Dictionary builder
0046 
0047 namespace internal {
0048 
0049 template <typename T, typename Enable = void>
0050 struct DictionaryValue {
0051   using type = typename T::c_type;
0052   using PhysicalType = T;
0053 };
0054 
0055 template <typename T>
0056 struct DictionaryValue<T, enable_if_base_binary<T>> {
0057   using type = std::string_view;
0058   using PhysicalType =
0059       typename std::conditional<std::is_same<typename T::offset_type, int32_t>::value,
0060                                 BinaryType, LargeBinaryType>::type;
0061 };
0062 
0063 template <typename T>
0064 struct DictionaryValue<T, enable_if_binary_view_like<T>> {
0065   using type = std::string_view;
0066   using PhysicalType = BinaryViewType;
0067 };
0068 
0069 template <typename T>
0070 struct DictionaryValue<T, enable_if_fixed_size_binary<T>> {
0071   using type = std::string_view;
0072   using PhysicalType = BinaryType;
0073 };
0074 
0075 class ARROW_EXPORT DictionaryMemoTable {
0076  public:
0077   DictionaryMemoTable(MemoryPool* pool, const std::shared_ptr<DataType>& type);
0078   DictionaryMemoTable(MemoryPool* pool, const std::shared_ptr<Array>& dictionary);
0079   ~DictionaryMemoTable();
0080 
0081   Status GetArrayData(int64_t start_offset, std::shared_ptr<ArrayData>* out);
0082 
0083   /// \brief Insert new memo values
0084   Status InsertValues(const Array& values);
0085 
0086   int32_t size() const;
0087 
0088   template <typename T>
0089   Status GetOrInsert(typename DictionaryValue<T>::type value, int32_t* out) {
0090     // We want to keep the DictionaryMemoTable implementation private, also we can't
0091     // use extern template classes because of compiler issues (MinGW?).  Instead,
0092     // we expose explicit function overrides for each supported physical type.
0093     const typename DictionaryValue<T>::PhysicalType* physical_type = NULLPTR;
0094     return GetOrInsert(physical_type, value, out);
0095   }
0096 
0097  private:
0098   Status GetOrInsert(const BooleanType*, bool value, int32_t* out);
0099   Status GetOrInsert(const Int8Type*, int8_t value, int32_t* out);
0100   Status GetOrInsert(const Int16Type*, int16_t value, int32_t* out);
0101   Status GetOrInsert(const Int32Type*, int32_t value, int32_t* out);
0102   Status GetOrInsert(const Int64Type*, int64_t value, int32_t* out);
0103   Status GetOrInsert(const UInt8Type*, uint8_t value, int32_t* out);
0104   Status GetOrInsert(const UInt16Type*, uint16_t value, int32_t* out);
0105   Status GetOrInsert(const UInt32Type*, uint32_t value, int32_t* out);
0106   Status GetOrInsert(const UInt64Type*, uint64_t value, int32_t* out);
0107   Status GetOrInsert(const DurationType*, int64_t value, int32_t* out);
0108   Status GetOrInsert(const TimestampType*, int64_t value, int32_t* out);
0109   Status GetOrInsert(const Date32Type*, int32_t value, int32_t* out);
0110   Status GetOrInsert(const Date64Type*, int64_t value, int32_t* out);
0111   Status GetOrInsert(const Time32Type*, int32_t value, int32_t* out);
0112   Status GetOrInsert(const Time64Type*, int64_t value, int32_t* out);
0113   Status GetOrInsert(const MonthDayNanoIntervalType*,
0114                      MonthDayNanoIntervalType::MonthDayNanos value, int32_t* out);
0115   Status GetOrInsert(const DayTimeIntervalType*,
0116                      DayTimeIntervalType::DayMilliseconds value, int32_t* out);
0117   Status GetOrInsert(const MonthIntervalType*, int32_t value, int32_t* out);
0118   Status GetOrInsert(const FloatType*, float value, int32_t* out);
0119   Status GetOrInsert(const DoubleType*, double value, int32_t* out);
0120 
0121   Status GetOrInsert(const BinaryType*, std::string_view value, int32_t* out);
0122   Status GetOrInsert(const LargeBinaryType*, std::string_view value, int32_t* out);
0123   Status GetOrInsert(const BinaryViewType*, std::string_view value, int32_t* out);
0124 
0125   class DictionaryMemoTableImpl;
0126   std::unique_ptr<DictionaryMemoTableImpl> impl_;
0127 };
0128 
0129 }  // namespace internal
0130 
0131 /// \addtogroup dictionary-builders
0132 ///
0133 /// @{
0134 
0135 namespace internal {
0136 
0137 /// \brief Array builder for created encoded DictionaryArray from
0138 /// dense array
0139 ///
0140 /// Unlike other builders, dictionary builder does not completely
0141 /// reset the state on Finish calls.
0142 template <typename BuilderType, typename T>
0143 class DictionaryBuilderBase : public ArrayBuilder {
0144  public:
0145   using TypeClass = DictionaryType;
0146   using Value = typename DictionaryValue<T>::type;
0147 
0148   // WARNING: the type given below is the value type, not the DictionaryType.
0149   // The DictionaryType is instantiated on the Finish() call.
0150   template <typename B = BuilderType, typename T1 = T>
0151   DictionaryBuilderBase(uint8_t start_int_size,
0152                         enable_if_t<std::is_base_of<AdaptiveIntBuilderBase, B>::value &&
0153                                         !is_fixed_size_binary_type<T1>::value,
0154                                     const std::shared_ptr<DataType>&>
0155                             value_type,
0156                         MemoryPool* pool = default_memory_pool(),
0157                         int64_t alignment = kDefaultBufferAlignment)
0158       : ArrayBuilder(pool, alignment),
0159         memo_table_(new internal::DictionaryMemoTable(pool, value_type)),
0160         delta_offset_(0),
0161         byte_width_(-1),
0162         indices_builder_(start_int_size, pool, alignment),
0163         value_type_(value_type) {}
0164 
0165   template <typename T1 = T>
0166   explicit DictionaryBuilderBase(
0167       enable_if_t<!is_fixed_size_binary_type<T1>::value, const std::shared_ptr<DataType>&>
0168           value_type,
0169       MemoryPool* pool = default_memory_pool(),
0170       int64_t alignment = kDefaultBufferAlignment)
0171       : ArrayBuilder(pool, alignment),
0172         memo_table_(new internal::DictionaryMemoTable(pool, value_type)),
0173         delta_offset_(0),
0174         byte_width_(-1),
0175         indices_builder_(pool, alignment),
0176         value_type_(value_type) {}
0177 
0178   template <typename T1 = T>
0179   explicit DictionaryBuilderBase(
0180       const std::shared_ptr<DataType>& index_type,
0181       enable_if_t<!is_fixed_size_binary_type<T1>::value, const std::shared_ptr<DataType>&>
0182           value_type,
0183       MemoryPool* pool = default_memory_pool(),
0184       int64_t alignment = kDefaultBufferAlignment)
0185       : ArrayBuilder(pool, alignment),
0186         memo_table_(new internal::DictionaryMemoTable(pool, value_type)),
0187         delta_offset_(0),
0188         byte_width_(-1),
0189         indices_builder_(index_type, pool, alignment),
0190         value_type_(value_type) {}
0191 
0192   template <typename B = BuilderType, typename T1 = T>
0193   DictionaryBuilderBase(uint8_t start_int_size,
0194                         enable_if_t<std::is_base_of<AdaptiveIntBuilderBase, B>::value &&
0195                                         is_fixed_size_binary_type<T1>::value,
0196                                     const std::shared_ptr<DataType>&>
0197                             value_type,
0198                         MemoryPool* pool = default_memory_pool(),
0199                         int64_t alignment = kDefaultBufferAlignment)
0200       : ArrayBuilder(pool, alignment),
0201         memo_table_(new internal::DictionaryMemoTable(pool, value_type)),
0202         delta_offset_(0),
0203         byte_width_(static_cast<const T1&>(*value_type).byte_width()),
0204         indices_builder_(start_int_size, pool, alignment),
0205         value_type_(value_type) {}
0206 
0207   template <typename T1 = T>
0208   explicit DictionaryBuilderBase(
0209       enable_if_fixed_size_binary<T1, const std::shared_ptr<DataType>&> value_type,
0210       MemoryPool* pool = default_memory_pool(),
0211       int64_t alignment = kDefaultBufferAlignment)
0212       : ArrayBuilder(pool, alignment),
0213         memo_table_(new internal::DictionaryMemoTable(pool, value_type)),
0214         delta_offset_(0),
0215         byte_width_(static_cast<const T1&>(*value_type).byte_width()),
0216         indices_builder_(pool, alignment),
0217         value_type_(value_type) {}
0218 
0219   template <typename T1 = T>
0220   explicit DictionaryBuilderBase(
0221       const std::shared_ptr<DataType>& index_type,
0222       enable_if_fixed_size_binary<T1, const std::shared_ptr<DataType>&> value_type,
0223       MemoryPool* pool = default_memory_pool(),
0224       int64_t alignment = kDefaultBufferAlignment)
0225       : ArrayBuilder(pool, alignment),
0226         memo_table_(new internal::DictionaryMemoTable(pool, value_type)),
0227         delta_offset_(0),
0228         byte_width_(static_cast<const T1&>(*value_type).byte_width()),
0229         indices_builder_(index_type, pool, alignment),
0230         value_type_(value_type) {}
0231 
0232   template <typename T1 = T>
0233   explicit DictionaryBuilderBase(
0234       enable_if_parameter_free<T1, MemoryPool*> pool = default_memory_pool())
0235       : DictionaryBuilderBase<BuilderType, T1>(TypeTraits<T1>::type_singleton(), pool) {}
0236 
0237   // This constructor doesn't check for errors. Use InsertMemoValues instead.
0238   explicit DictionaryBuilderBase(const std::shared_ptr<Array>& dictionary,
0239                                  MemoryPool* pool = default_memory_pool(),
0240                                  int64_t alignment = kDefaultBufferAlignment)
0241       : ArrayBuilder(pool, alignment),
0242         memo_table_(new internal::DictionaryMemoTable(pool, dictionary)),
0243         delta_offset_(0),
0244         byte_width_(-1),
0245         indices_builder_(pool, alignment),
0246         value_type_(dictionary->type()) {}
0247 
0248   ~DictionaryBuilderBase() override = default;
0249 
0250   /// \brief The current number of entries in the dictionary
0251   int64_t dictionary_length() const { return memo_table_->size(); }
0252 
0253   /// \brief The value byte width (for FixedSizeBinaryType)
0254   template <typename T1 = T>
0255   enable_if_fixed_size_binary<T1, int32_t> byte_width() const {
0256     return byte_width_;
0257   }
0258 
0259   /// \brief Append a scalar value
0260   Status Append(Value value) {
0261     ARROW_RETURN_NOT_OK(Reserve(1));
0262 
0263     int32_t memo_index;
0264     ARROW_RETURN_NOT_OK(memo_table_->GetOrInsert<T>(value, &memo_index));
0265     ARROW_RETURN_NOT_OK(indices_builder_.Append(memo_index));
0266     length_ += 1;
0267 
0268     return Status::OK();
0269   }
0270 
0271   /// \brief Append a fixed-width string (only for FixedSizeBinaryType)
0272   template <typename T1 = T>
0273   enable_if_fixed_size_binary<T1, Status> Append(const uint8_t* value) {
0274     return Append(std::string_view(reinterpret_cast<const char*>(value), byte_width_));
0275   }
0276 
0277   /// \brief Append a fixed-width string (only for FixedSizeBinaryType)
0278   template <typename T1 = T>
0279   enable_if_fixed_size_binary<T1, Status> Append(const char* value) {
0280     return Append(std::string_view(value, byte_width_));
0281   }
0282 
0283   /// \brief Append a string (only for binary types)
0284   template <typename T1 = T>
0285   enable_if_binary_like<T1, Status> Append(const uint8_t* value, int32_t length) {
0286     return Append(reinterpret_cast<const char*>(value), length);
0287   }
0288 
0289   /// \brief Append a string (only for binary types)
0290   template <typename T1 = T>
0291   enable_if_binary_like<T1, Status> Append(const char* value, int32_t length) {
0292     return Append(std::string_view(value, length));
0293   }
0294 
0295   /// \brief Append a string (only for string types)
0296   template <typename T1 = T>
0297   enable_if_string_like<T1, Status> Append(const char* value, int32_t length) {
0298     return Append(std::string_view(value, length));
0299   }
0300 
0301   /// \brief Append a decimal (only for Decimal32/64/128/256 Type)
0302   template <typename T1 = T, typename CType = typename TypeTraits<T1>::CType>
0303   enable_if_decimal<T1, Status> Append(const CType& value) {
0304     auto bytes = value.ToBytes();
0305     return Append(bytes.data(), static_cast<int32_t>(bytes.size()));
0306   }
0307 
0308   /// \brief Append a scalar null value
0309   Status AppendNull() final {
0310     length_ += 1;
0311     null_count_ += 1;
0312 
0313     return indices_builder_.AppendNull();
0314   }
0315 
0316   Status AppendNulls(int64_t length) final {
0317     length_ += length;
0318     null_count_ += length;
0319 
0320     return indices_builder_.AppendNulls(length);
0321   }
0322 
0323   Status AppendEmptyValue() final {
0324     length_ += 1;
0325 
0326     return indices_builder_.AppendEmptyValue();
0327   }
0328 
0329   Status AppendEmptyValues(int64_t length) final {
0330     length_ += length;
0331 
0332     return indices_builder_.AppendEmptyValues(length);
0333   }
0334 
0335   Status AppendScalar(const Scalar& scalar, int64_t n_repeats) override {
0336     if (!scalar.is_valid) return AppendNulls(n_repeats);
0337 
0338     const auto& dict_ty = internal::checked_cast<const DictionaryType&>(*scalar.type);
0339     const DictionaryScalar& dict_scalar =
0340         internal::checked_cast<const DictionaryScalar&>(scalar);
0341     const auto& dict = internal::checked_cast<const typename TypeTraits<T>::ArrayType&>(
0342         *dict_scalar.value.dictionary);
0343     ARROW_RETURN_NOT_OK(Reserve(n_repeats));
0344     switch (dict_ty.index_type()->id()) {
0345       case Type::UINT8:
0346         return AppendScalarImpl<UInt8Type>(dict, *dict_scalar.value.index, n_repeats);
0347       case Type::INT8:
0348         return AppendScalarImpl<Int8Type>(dict, *dict_scalar.value.index, n_repeats);
0349       case Type::UINT16:
0350         return AppendScalarImpl<UInt16Type>(dict, *dict_scalar.value.index, n_repeats);
0351       case Type::INT16:
0352         return AppendScalarImpl<Int16Type>(dict, *dict_scalar.value.index, n_repeats);
0353       case Type::UINT32:
0354         return AppendScalarImpl<UInt32Type>(dict, *dict_scalar.value.index, n_repeats);
0355       case Type::INT32:
0356         return AppendScalarImpl<Int32Type>(dict, *dict_scalar.value.index, n_repeats);
0357       case Type::UINT64:
0358         return AppendScalarImpl<UInt64Type>(dict, *dict_scalar.value.index, n_repeats);
0359       case Type::INT64:
0360         return AppendScalarImpl<Int64Type>(dict, *dict_scalar.value.index, n_repeats);
0361       default:
0362         return Status::TypeError("Invalid index type: ", dict_ty);
0363     }
0364     return Status::OK();
0365   }
0366 
0367   Status AppendScalars(const ScalarVector& scalars) override {
0368     for (const auto& scalar : scalars) {
0369       ARROW_RETURN_NOT_OK(AppendScalar(*scalar, /*n_repeats=*/1));
0370     }
0371     return Status::OK();
0372   }
0373 
0374   Status AppendArraySlice(const ArraySpan& array, int64_t offset, int64_t length) final {
0375     // Visit the indices and insert the unpacked values.
0376     const auto& dict_ty = internal::checked_cast<const DictionaryType&>(*array.type);
0377     // See if possible to avoid using ToArrayData here
0378     const typename TypeTraits<T>::ArrayType dict(array.dictionary().ToArrayData());
0379     ARROW_RETURN_NOT_OK(Reserve(length));
0380     switch (dict_ty.index_type()->id()) {
0381       case Type::UINT8:
0382         return AppendArraySliceImpl<uint8_t>(dict, array, offset, length);
0383       case Type::INT8:
0384         return AppendArraySliceImpl<int8_t>(dict, array, offset, length);
0385       case Type::UINT16:
0386         return AppendArraySliceImpl<uint16_t>(dict, array, offset, length);
0387       case Type::INT16:
0388         return AppendArraySliceImpl<int16_t>(dict, array, offset, length);
0389       case Type::UINT32:
0390         return AppendArraySliceImpl<uint32_t>(dict, array, offset, length);
0391       case Type::INT32:
0392         return AppendArraySliceImpl<int32_t>(dict, array, offset, length);
0393       case Type::UINT64:
0394         return AppendArraySliceImpl<uint64_t>(dict, array, offset, length);
0395       case Type::INT64:
0396         return AppendArraySliceImpl<int64_t>(dict, array, offset, length);
0397       default:
0398         return Status::TypeError("Invalid index type: ", dict_ty);
0399     }
0400     return Status::OK();
0401   }
0402 
0403   /// \brief Insert values into the dictionary's memo, but do not append any
0404   /// indices. Can be used to initialize a new builder with known dictionary
0405   /// values
0406   /// \param[in] values dictionary values to add to memo. Type must match
0407   /// builder type
0408   Status InsertMemoValues(const Array& values) {
0409     return memo_table_->InsertValues(values);
0410   }
0411 
0412   /// \brief Append a whole dense array to the builder
0413   template <typename T1 = T>
0414   enable_if_t<!is_fixed_size_binary_type<T1>::value, Status> AppendArray(
0415       const Array& array) {
0416     using ArrayType = typename TypeTraits<T>::ArrayType;
0417 
0418 #ifndef NDEBUG
0419     ARROW_RETURN_NOT_OK(ArrayBuilder::CheckArrayType(
0420         value_type_, array, "Wrong value type of array to be appended"));
0421 #endif
0422 
0423     const auto& concrete_array = static_cast<const ArrayType&>(array);
0424     for (int64_t i = 0; i < array.length(); i++) {
0425       if (array.IsNull(i)) {
0426         ARROW_RETURN_NOT_OK(AppendNull());
0427       } else {
0428         ARROW_RETURN_NOT_OK(Append(concrete_array.GetView(i)));
0429       }
0430     }
0431     return Status::OK();
0432   }
0433 
0434   template <typename T1 = T>
0435   enable_if_fixed_size_binary<T1, Status> AppendArray(const Array& array) {
0436 #ifndef NDEBUG
0437     ARROW_RETURN_NOT_OK(ArrayBuilder::CheckArrayType(
0438         value_type_, array, "Wrong value type of array to be appended"));
0439 #endif
0440 
0441     const auto& concrete_array = static_cast<const FixedSizeBinaryArray&>(array);
0442     for (int64_t i = 0; i < array.length(); i++) {
0443       if (array.IsNull(i)) {
0444         ARROW_RETURN_NOT_OK(AppendNull());
0445       } else {
0446         ARROW_RETURN_NOT_OK(Append(concrete_array.GetValue(i)));
0447       }
0448     }
0449     return Status::OK();
0450   }
0451 
0452   void Reset() override {
0453     // Perform a partial reset. Call ResetFull to also reset the accumulated
0454     // dictionary values
0455     ArrayBuilder::Reset();
0456     indices_builder_.Reset();
0457   }
0458 
0459   /// \brief Reset and also clear accumulated dictionary values in memo table
0460   void ResetFull() {
0461     Reset();
0462     memo_table_.reset(new internal::DictionaryMemoTable(pool_, value_type_));
0463   }
0464 
0465   Status Resize(int64_t capacity) override {
0466     ARROW_RETURN_NOT_OK(CheckCapacity(capacity));
0467     capacity = std::max(capacity, kMinBuilderCapacity);
0468     ARROW_RETURN_NOT_OK(indices_builder_.Resize(capacity));
0469     capacity_ = indices_builder_.capacity();
0470     return Status::OK();
0471   }
0472 
0473   /// \brief Return dictionary indices and a delta dictionary since the last
0474   /// time that Finish or FinishDelta were called, and reset state of builder
0475   /// (except the memo table)
0476   Status FinishDelta(std::shared_ptr<Array>* out_indices,
0477                      std::shared_ptr<Array>* out_delta) {
0478     std::shared_ptr<ArrayData> indices_data;
0479     std::shared_ptr<ArrayData> delta_data;
0480     ARROW_RETURN_NOT_OK(FinishWithDictOffset(delta_offset_, &indices_data, &delta_data));
0481     *out_indices = MakeArray(indices_data);
0482     *out_delta = MakeArray(delta_data);
0483     return Status::OK();
0484   }
0485 
0486   /// \cond FALSE
0487   using ArrayBuilder::Finish;
0488   /// \endcond
0489 
0490   Status Finish(std::shared_ptr<DictionaryArray>* out) { return FinishTyped(out); }
0491 
0492   std::shared_ptr<DataType> type() const override {
0493     return ::arrow::dictionary(indices_builder_.type(), value_type_);
0494   }
0495 
0496  protected:
0497   template <typename c_type>
0498   Status AppendArraySliceImpl(const typename TypeTraits<T>::ArrayType& dict,
0499                               const ArraySpan& array, int64_t offset, int64_t length) {
0500     const c_type* values = array.GetValues<c_type>(1) + offset;
0501     return VisitBitBlocks(
0502         array.buffers[0].data, array.offset + offset, length,
0503         [&](const int64_t position) {
0504           const int64_t index = static_cast<int64_t>(values[position]);
0505           if (dict.IsValid(index)) {
0506             return Append(dict.GetView(index));
0507           }
0508           return AppendNull();
0509         },
0510         [&]() { return AppendNull(); });
0511   }
0512 
0513   template <typename IndexType>
0514   Status AppendScalarImpl(const typename TypeTraits<T>::ArrayType& dict,
0515                           const Scalar& index_scalar, int64_t n_repeats) {
0516     using ScalarType = typename TypeTraits<IndexType>::ScalarType;
0517     const auto index = internal::checked_cast<const ScalarType&>(index_scalar).value;
0518     if (index_scalar.is_valid && dict.IsValid(index)) {
0519       const auto& value = dict.GetView(index);
0520       for (int64_t i = 0; i < n_repeats; i++) {
0521         ARROW_RETURN_NOT_OK(Append(value));
0522       }
0523       return Status::OK();
0524     }
0525     return AppendNulls(n_repeats);
0526   }
0527 
0528   Status FinishInternal(std::shared_ptr<ArrayData>* out) override {
0529     std::shared_ptr<ArrayData> dictionary;
0530     ARROW_RETURN_NOT_OK(FinishWithDictOffset(/*offset=*/0, out, &dictionary));
0531 
0532     // Set type of array data to the right dictionary type
0533     (*out)->type = type();
0534     (*out)->dictionary = dictionary;
0535     return Status::OK();
0536   }
0537 
0538   Status FinishWithDictOffset(int64_t dict_offset,
0539                               std::shared_ptr<ArrayData>* out_indices,
0540                               std::shared_ptr<ArrayData>* out_dictionary) {
0541     // Finalize indices array
0542     ARROW_RETURN_NOT_OK(indices_builder_.FinishInternal(out_indices));
0543 
0544     // Generate dictionary array from hash table contents
0545     ARROW_RETURN_NOT_OK(memo_table_->GetArrayData(dict_offset, out_dictionary));
0546     delta_offset_ = memo_table_->size();
0547 
0548     // Update internals for further uses of this DictionaryBuilder
0549     ArrayBuilder::Reset();
0550     return Status::OK();
0551   }
0552 
0553   std::unique_ptr<DictionaryMemoTable> memo_table_;
0554 
0555   // The size of the dictionary memo at last invocation of Finish, to use in
0556   // FinishDelta for computing dictionary deltas
0557   int32_t delta_offset_;
0558 
0559   // Only used for FixedSizeBinaryType
0560   int32_t byte_width_;
0561 
0562   BuilderType indices_builder_;
0563   std::shared_ptr<DataType> value_type_;
0564 };
0565 
0566 template <typename BuilderType>
0567 class DictionaryBuilderBase<BuilderType, NullType> : public ArrayBuilder {
0568  public:
0569   template <typename B = BuilderType>
0570   DictionaryBuilderBase(
0571       enable_if_t<std::is_base_of<AdaptiveIntBuilderBase, B>::value, uint8_t>
0572           start_int_size,
0573       const std::shared_ptr<DataType>& value_type,
0574       MemoryPool* pool = default_memory_pool())
0575       : ArrayBuilder(pool), indices_builder_(start_int_size, pool) {}
0576 
0577   explicit DictionaryBuilderBase(const std::shared_ptr<DataType>& value_type,
0578                                  MemoryPool* pool = default_memory_pool())
0579       : ArrayBuilder(pool), indices_builder_(pool) {}
0580 
0581   explicit DictionaryBuilderBase(const std::shared_ptr<DataType>& index_type,
0582                                  const std::shared_ptr<DataType>& value_type,
0583                                  MemoryPool* pool = default_memory_pool())
0584       : ArrayBuilder(pool), indices_builder_(index_type, pool) {}
0585 
0586   template <typename B = BuilderType>
0587   explicit DictionaryBuilderBase(
0588       enable_if_t<std::is_base_of<AdaptiveIntBuilderBase, B>::value, uint8_t>
0589           start_int_size,
0590       MemoryPool* pool = default_memory_pool())
0591       : ArrayBuilder(pool), indices_builder_(start_int_size, pool) {}
0592 
0593   explicit DictionaryBuilderBase(MemoryPool* pool = default_memory_pool())
0594       : ArrayBuilder(pool), indices_builder_(pool) {}
0595 
0596   explicit DictionaryBuilderBase(const std::shared_ptr<Array>& dictionary,
0597                                  MemoryPool* pool = default_memory_pool())
0598       : ArrayBuilder(pool), indices_builder_(pool) {}
0599 
0600   /// \brief Append a scalar null value
0601   Status AppendNull() final {
0602     length_ += 1;
0603     null_count_ += 1;
0604 
0605     return indices_builder_.AppendNull();
0606   }
0607 
0608   Status AppendNulls(int64_t length) final {
0609     length_ += length;
0610     null_count_ += length;
0611 
0612     return indices_builder_.AppendNulls(length);
0613   }
0614 
0615   Status AppendEmptyValue() final {
0616     length_ += 1;
0617 
0618     return indices_builder_.AppendEmptyValue();
0619   }
0620 
0621   Status AppendEmptyValues(int64_t length) final {
0622     length_ += length;
0623 
0624     return indices_builder_.AppendEmptyValues(length);
0625   }
0626 
0627   /// \brief Append a whole dense array to the builder
0628   Status AppendArray(const Array& array) {
0629 #ifndef NDEBUG
0630     ARROW_RETURN_NOT_OK(ArrayBuilder::CheckArrayType(
0631         Type::NA, array, "Wrong value type of array to be appended"));
0632 #endif
0633     for (int64_t i = 0; i < array.length(); i++) {
0634       ARROW_RETURN_NOT_OK(AppendNull());
0635     }
0636     return Status::OK();
0637   }
0638 
0639   Status Resize(int64_t capacity) override {
0640     ARROW_RETURN_NOT_OK(CheckCapacity(capacity));
0641     capacity = std::max(capacity, kMinBuilderCapacity);
0642 
0643     ARROW_RETURN_NOT_OK(indices_builder_.Resize(capacity));
0644     capacity_ = indices_builder_.capacity();
0645     return Status::OK();
0646   }
0647 
0648   Status FinishInternal(std::shared_ptr<ArrayData>* out) override {
0649     ARROW_RETURN_NOT_OK(indices_builder_.FinishInternal(out));
0650     (*out)->type = dictionary((*out)->type, null());
0651     (*out)->dictionary = NullArray(0).data();
0652     return Status::OK();
0653   }
0654 
0655   /// \cond FALSE
0656   using ArrayBuilder::Finish;
0657   /// \endcond
0658 
0659   Status Finish(std::shared_ptr<DictionaryArray>* out) { return FinishTyped(out); }
0660 
0661   std::shared_ptr<DataType> type() const override {
0662     return ::arrow::dictionary(indices_builder_.type(), null());
0663   }
0664 
0665  protected:
0666   BuilderType indices_builder_;
0667 };
0668 
0669 }  // namespace internal
0670 
0671 /// \brief A DictionaryArray builder that uses AdaptiveIntBuilder to return the
0672 /// smallest index size that can accommodate the dictionary indices
0673 template <typename T>
0674 class DictionaryBuilder : public internal::DictionaryBuilderBase<AdaptiveIntBuilder, T> {
0675  public:
0676   using BASE = internal::DictionaryBuilderBase<AdaptiveIntBuilder, T>;
0677   using BASE::BASE;
0678 
0679   /// \brief Append dictionary indices directly without modifying memo
0680   ///
0681   /// NOTE: Experimental API
0682   Status AppendIndices(const int64_t* values, int64_t length,
0683                        const uint8_t* valid_bytes = NULLPTR) {
0684     int64_t null_count_before = this->indices_builder_.null_count();
0685     ARROW_RETURN_NOT_OK(this->indices_builder_.AppendValues(values, length, valid_bytes));
0686     this->capacity_ = this->indices_builder_.capacity();
0687     this->length_ += length;
0688     this->null_count_ += this->indices_builder_.null_count() - null_count_before;
0689     return Status::OK();
0690   }
0691 };
0692 
0693 /// \brief A DictionaryArray builder that always returns int32 dictionary
0694 /// indices so that data cast to dictionary form will have a consistent index
0695 /// type, e.g. for creating a ChunkedArray
0696 template <typename T>
0697 class Dictionary32Builder : public internal::DictionaryBuilderBase<Int32Builder, T> {
0698  public:
0699   using BASE = internal::DictionaryBuilderBase<Int32Builder, T>;
0700   using BASE::BASE;
0701 
0702   /// \brief Append dictionary indices directly without modifying memo
0703   ///
0704   /// NOTE: Experimental API
0705   Status AppendIndices(const int32_t* values, int64_t length,
0706                        const uint8_t* valid_bytes = NULLPTR) {
0707     int64_t null_count_before = this->indices_builder_.null_count();
0708     ARROW_RETURN_NOT_OK(this->indices_builder_.AppendValues(values, length, valid_bytes));
0709     this->capacity_ = this->indices_builder_.capacity();
0710     this->length_ += length;
0711     this->null_count_ += this->indices_builder_.null_count() - null_count_before;
0712     return Status::OK();
0713   }
0714 };
0715 
0716 // ----------------------------------------------------------------------
0717 // Binary / Unicode builders
0718 // (compatibility aliases; those used to be derived classes with additional
0719 //  Append() overloads, but they have been folded into DictionaryBuilderBase)
0720 
0721 using BinaryDictionaryBuilder = DictionaryBuilder<BinaryType>;
0722 using StringDictionaryBuilder = DictionaryBuilder<StringType>;
0723 using BinaryDictionary32Builder = Dictionary32Builder<BinaryType>;
0724 using StringDictionary32Builder = Dictionary32Builder<StringType>;
0725 
0726 /// @}
0727 
0728 }  // namespace arrow