You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/10/07 04:25:45 UTC

[GitHub] [arrow] rok opened a new pull request, #14341: ARROW-17619: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

rok opened a new pull request, #14341:
URL: https://github.com/apache/arrow/pull/14341

   This is to add DELTA_BYTE_ARRAY encoder.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1159713114


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3083,242 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, nullptr)) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          buffer_, ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);

Review Comment:
   `PutBinaryArray` now passes, but could perhaps be optimized:
   ```cpp
   if (suffix_length == 0) {
     const auto suffix_ptr = reinterpret_cast<const uint8_t*>("");
     const ByteArray suffix(suffix_length, suffix_ptr);
     suffix_encoder_.Put(&suffix, 1);
     return Status::OK();
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1162196583


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3196,6 +3469,43 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) {
+    return GetInternal(buffer, max_values);
+  }
+  int Decode(FixedLenByteArray* buffer, int max_values) override {
+    int decoded_values_size = max_values;
+    if (MultiplyWithOverflow(decoded_values_size,
+                             descr_->type_length() * sizeof(ByteArray),
+                             &decoded_values_size)) {
+      throw ParquetException("excess expansion in DELTA_LENGTH_BYTE_ARRAY");
+    }
+    std::vector<uint8_t> decode_values(decoded_values_size);

Review Comment:
   Switched to:
   ```cpp
       ArrowPoolVector<uint8_t> decode_values(decoded_values_size,
                                              ::arrow::stl::allocator<uint8_t>(pool_));
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1502884543

   > If there are no objections I'd like to merge this before 12.0.0 window closes tomorrow (Tuesday).
   
   There is no need to rush this and I think it can wait for a more comprehensive review. There have been enough delicate issues with the DELTA decoders/encoders in the past, to exercise a bit more restraint.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1502529329

   Thanks for the review @mapleFU! If CI passes and there's no further comments I'll merge.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1145570380


##########
cpp/src/parquet/encoding.cc:
##########
@@ -2900,8 +2929,222 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+ public:
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr, MemoryPool* pool)
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY,
+                    pool = ::arrow::default_memory_pool()),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+
+    // TODO(rok): optimize using ArrowPoolVector<int32_t> prefix_lengths(num_values);
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          uint32_t previous_len = 0;
+          const ByteArray src{view};

Review Comment:
   `suffix_encoder_` takes `ByteArray` so we need to change type. I'm not sure how else to do it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1146207467


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3062,226 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+    uint32_t previous_len = byte_width;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {

Review Comment:
   Removed PREDICT.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1482212682

   > Seems the skeleton look great now. I think we can extract `Put` because now we have 4 `Put(...)` implemention now...
   
   Will do first thing tomorrow!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1149414112


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,180 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;

Review Comment:
   Which test would failed here? I pull and run test, `parquet-internal-test` and `parquet-arrow-internal-test` passed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1149494634


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,180 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = j;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          const uint8_t* suffix_ptr = src.ptr + j;
+          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+          last_value_view =
+              string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+          // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  int i = 0;
+  while (i < num_values) {
+    // Convert to ByteArray so we can pass to the suffix_encoder_.
+    auto value = reinterpret_cast<const ByteArray*>(&src[i]);
+    if (ARROW_PREDICT_FALSE(value->len >= kMaxByteArraySize)) {
+      throw Status::Invalid("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value->ptr), value->len};
+    uint32_t j = 0;
+    while (j < std::min(value->len, static_cast<uint32_t>(last_value_view.length()))) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
+    prefix_lengths[i] = j;
+    const uint8_t* suffix_ptr = value->ptr + j;
+    const uint32_t suffix_length = static_cast<uint32_t>(value->len - j);
+    last_value_view =
+        string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+    // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+    const ByteArray suffix(suffix_length, suffix_ptr);
+    suffix_encoder_.Put(&suffix, 1);
+    i++;
+  }
+  prefix_length_encoder_.Put(prefix_lengths.data(), num_values);
+  last_value_ = last_value_view;
+}
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const ::arrow::Array& values) {
+  if (::arrow::is_binary_like(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::BinaryArray&>(values));
+  } else if (::arrow::is_large_binary_like(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::LargeBinaryArray&>(values));
+  } else if (::arrow::is_fixed_size_binary(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::FixedSizeBinaryArray&>(values));
+  } else {
+    throw ParquetException("Only BaseBinaryArray and subclasses supported");
+  }
+}
+
+template <typename DType>
+std::shared_ptr<Buffer> DeltaByteArrayEncoder<DType>::FlushValues() {
+  PARQUET_THROW_NOT_OK(sink_.Resize(EstimatedDataEncodedSize(), false));
+
+  std::shared_ptr<Buffer> prefix_lengths = prefix_length_encoder_.FlushValues();
+  PARQUET_THROW_NOT_OK(sink_.Append(prefix_lengths->data(), prefix_lengths->size()));
+
+  std::shared_ptr<Buffer> suffixes = suffix_encoder_.FlushValues();
+  PARQUET_THROW_NOT_OK(sink_.Append(suffixes->data(), suffixes->size()));
+
+  std::shared_ptr<Buffer> buffer;
+  PARQUET_THROW_NOT_OK(sink_.Finish(&buffer, true));
+  return buffer;

Review Comment:
   ```suggestion
     last_value_.clear();
     return buffer;
   ```



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,180 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = j;

Review Comment:
   ```suggestion
             previous_len = view.size();
   ```



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,159 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  virtual void CheckRoundtrip() {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits, int64_t valid_bits_offset) {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+// TODO: add FLBAType and Decimal type tests
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 2));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_prob*/ 0));
+
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 200));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.1));
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPut) {
+  const int64_t size = 50;
+  const int32_t min_length = 0;
+  const int32_t max_length = 10;
+  const int32_t num_unique = 10;
+  const double null_probability = 0.25;
+  auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  ::arrow::random::RandomArrayGenerator rag(42);
+  auto values = rag.String(0, min_length, max_length, null_probability);
+  CheckSeed(values);
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    rag = ::arrow::random::RandomArrayGenerator(seed);
+
+    values = rag.String(size, min_length, max_length, null_probability);
+    CheckSeed(values);
+
+    values =
+        rag.BinaryWithRepeats(size, num_unique, min_length, max_length, null_probability);
+    CheckSeed(values);
+  }
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPutFixedLength) {
+  const int64_t size = 50;
+  const double null_probability = 0.25;
+  ::arrow::random::RandomArrayGenerator rag(0);
+  auto encoder = MakeTypedEncoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<FLBAType>::Accumulator acc(values->type());
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    for (auto length : {0, 10, 100, 1000}) {
+      rag = ::arrow::random::RandomArrayGenerator(seed);
+      auto values = rag.FixedSizeBinary(size, length, null_probability);
+      CheckSeed(values);
+    }
+  }
+}
+

Review Comment:
   I'd like to add a test here, it's a bit like `DeltaLengthByteArrayEncodingAdHoc.ArrowDirectPut`, but I didn't finish it.
   
   ```suggestion
   TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
     auto CheckEncode = [](std::shared_ptr<::arrow::Array> values,
                           std::shared_ptr<::arrow::Array> prefix_lengths,
                           std::shared_ptr<::arrow::Array> suffix_lengths, std::string_view value)  {
       auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
       ASSERT_NO_THROW(encoder->Put(*values));
       auto buf = encoder->FlushValues();
   
       auto prefix_lengths_encoder = MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
       ASSERT_NO_THROW(prefix_lengths_encoder->Put(*prefix_lengths));
       auto prefix_lengths_buf = prefix_lengths_encoder->FlushValues();
   
       auto encoded_prefix_lengths_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
   
       auto suffix_lengths_encoder = MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
       ASSERT_NO_THROW(suffix_lengths_encoder->Put(*suffix_lengths));
       auto suffix_lengths_buf = suffix_lengths_encoder->FlushValues();
   
       auto encoded_values_buf = SliceBuffer(buf, prefix_lengths_buf->size() + suffix_lengths_buf->size());
   
       auto encoded_prefix_length_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
       EXPECT_TRUE(prefix_lengths_buf->Equals(*encoded_prefix_length_buf));
       auto encoded_suffix_length_buf = SliceBuffer(buf, prefix_lengths_buf->size(), suffix_lengths_buf->size());
       EXPECT_TRUE(suffix_lengths_buf->Equals(*encoded_suffix_length_buf));
       EXPECT_EQ(value, encoded_values_buf->ToString());
     };
   
     auto values = R"(["axis", "axle", "babble", "babyhood"])";
     auto prefix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([0, 2, 0, 3])");
     auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([4, 2, 6, 5])");
   
     CheckEncode(::arrow::ArrayFromJSON(::arrow::utf8(), values), prefix_lengths, suffix_lengths, "axislebabbleyhood");
     CheckEncode(::arrow::ArrayFromJSON(::arrow::large_utf8(), values), prefix_lengths, suffix_lengths, "axislebabbleyhood");
     CheckEncode(::arrow::ArrayFromJSON(::arrow::binary(), values), prefix_lengths, suffix_lengths, "axislebabbleyhood");
     CheckEncode(::arrow::ArrayFromJSON(::arrow::large_binary(), values), prefix_lengths, suffix_lengths, "axislebabbleyhood");
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1198419971


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,243 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw ParquetException("Put not implemented for " + this->descr_->ToString());
+}
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i++) {
+    // Convert to ByteArray, so we can pass to the suffix_encoder_.
+    const ByteArray value = src[i];
+    if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+      throw ParquetException("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value.ptr),
+                            static_cast<uint32_t>(value.len)};
+    uint32_t j = 0;
+    const uint32_t common_length =
+        std::min(value.len, static_cast<uint32_t>(last_value_view.length()));
+    while (j < common_length) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
+    last_value_view = view;
+    prefix_lengths[i] = j;
+    const auto suffix_length = static_cast<uint32_t>(value.len - j);
+
+    if (suffix_length == 0) {
+      suffix_encoder_.Put(&kEmpty, 1);
+      continue;
+    }
+    const uint8_t* suffix_ptr = value.ptr + j;
+    // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+    const ByteArray suffix(suffix_length, suffix_ptr);
+    suffix_encoder_.Put(&suffix, 1);
+  }
+  prefix_length_encoder_.Put(prefix_lengths.data(), num_values);
+  last_value_ = last_value_view;
+}
+
+template <>
+void DeltaByteArrayEncoder<FLBAType>::Put(const FLBA* src, int num_values) {

Review Comment:
   That works indeed. Refactored the two `Put` functions to `PutInternal` (https://github.com/apache/arrow/pull/14341/commits/1d9b54a618e78d5077f1f81ea8c80f39511e4371).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] wgtmac commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1625666937

   There seems to be obvious issues from the ASAN report (https://github.com/apache/arrow/actions/runs/5481124113/jobs/9985061685?pr=14341
   ). Could you fix them first?
   
   ```
   FAILED: src/parquet/CMakeFiles/parquet-internals-test.dir/encoding_test.cc.o 
   /arrow/cpp/src/parquet/encoding_test.cc:1954:35: error: implicit conversion loses integer precision: 'long' to 'int' [-Werror,-Wshorten-64-to-32]
       num_values_ = array->length() - array->null_count();
                   ~ ~~~~~~~~~~~~~~~~^~~~~~~~~~~~~~~~~~~~~
   /arrow/cpp/src/parquet/encoding_test.cc:1959:5: note: in instantiation of member function 'parquet::test::TestDeltaByteArrayEncoding<parquet::PhysicalType<parquet::Type::FIXED_LEN_BYTE_ARRAY>>::InitData' requested here
       InitData(nvalues, null_probability);
       ^
   /arrow/cpp/src/parquet/encoding_test.cc:2022:33: note: in instantiation of member function 'parquet::test::TestDeltaByteArrayEncoding<parquet::PhysicalType<parquet::Type::FIXED_LEN_BYTE_ARRAY>>::Execute' requested here
     ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
                                   ^
   /build/cpp/googletest_ep-prefix/include/gtest/internal/gtest-internal.h:472:44: note: in instantiation of member function 'parquet::test::TestDeltaByteArrayEncoding_BasicRoundTrip_Test<parquet::PhysicalType<parquet::Type::FIXED_LEN_BYTE_ARRAY>>::TestBody' requested here
     Test* CreateTest() override { return new TestClass; }
                                              ^
   /build/cpp/googletest_ep-prefix/include/gtest/internal/gtest-internal.h:740:13: note: in instantiation of member function 'testing::internal::TestFactoryImpl<parquet::test::TestDeltaByteArrayEncoding_BasicRoundTrip_Test<parquet::PhysicalType<parquet::Type::FIXED_LEN_BYTE_ARRAY>>>::CreateTest' requested here
           new TestFactoryImpl<TestClass>);
               ^
   /build/cpp/googletest_ep-prefix/include/gtest/internal/gtest-internal.h:744:57: note: in instantiation of member function 'testing::internal::TypeParameterizedTest<parquet::test::TestDeltaByteArrayEncoding, testing::internal::TemplateSel<parquet::test::TestDeltaByteArrayEncoding_BasicRoundTrip_Test>, testing::internal::Types<parquet::PhysicalType<parquet::Type::FIXED_LEN_BYTE_ARRAY>>>::Register' requested here
                                    typename Types::Tail>::Register(prefix,
                                                           ^
   /arrow/cpp/src/parquet/encoding_test.cc:2021:1: note: in instantiation of member function 'testing::internal::TypeParameterizedTest<parquet::test::TestDeltaByteArrayEncoding, testing::internal::TemplateSel<parquet::test::TestDeltaByteArrayEncoding_BasicRoundTrip_Test>, testing::internal::Types<parquet::PhysicalType<parquet::Type::BYTE_ARRAY>, parquet::PhysicalType<parquet::Type::FIXED_LEN_BYTE_ARRAY>>>::Register' requested here
   TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
   ^
   /build/cpp/googletest_ep-prefix/include/gtest/gtest-typed-test.h:212:27: note: expanded from macro 'TYPED_TEST'
                 CaseName)>::Register("",                                        \
                             ^
   /arrow/cpp/src/parquet/encoding_test.cc:1954:35: error: implicit conversion loses integer precision: 'long' to 'int' [-Werror,-Wshorten-64-to-32]
       num_values_ = array->length() - array->null_count();
                   ~ ~~~~~~~~~~~~~~~~^~~~~~~~~~~~~~~~~~~~~
   /arrow/cpp/src/parquet/encoding_test.cc:1959:5: note: in instantiation of member function 'parquet::test::TestDeltaByteArrayEncoding<parquet::PhysicalType<parquet::Type::BYTE_ARRAY>>::InitData' requested here
       InitData(nvalues, null_probability);
       ^
   /arrow/cpp/src/parquet/encoding_test.cc:2022:33: note: in instantiation of member function 'parquet::test::TestDeltaByteArrayEncoding<parquet::PhysicalType<parquet::Type::BYTE_ARRAY>>::Execute' requested here
     ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
                                   ^
   /build/cpp/googletest_ep-prefix/include/gtest/internal/gtest-internal.h:472:44: note: in instantiation of member function 'parquet::test::TestDeltaByteArrayEncoding_BasicRoundTrip_Test<parquet::PhysicalType<parquet::Type::BYTE_ARRAY>>::TestBody' requested here
     Test* CreateTest() override { return new TestClass; }
                                              ^
   /build/cpp/googletest_ep-prefix/include/gtest/internal/gtest-internal.h:740:13: note: in instantiation of member function 'testing::internal::TestFactoryImpl<parquet::test::TestDeltaByteArrayEncoding_BasicRoundTrip_Test<parquet::PhysicalType<parquet::Type::BYTE_ARRAY>>>::CreateTest' requested here
           new TestFactoryImpl<TestClass>);
               ^
   /arrow/cpp/src/parquet/encoding_test.cc:2021:1: note: in instantiation of member function 'testing::internal::TypeParameterizedTest<parquet::test::TestDeltaByteArrayEncoding, testing::internal::TemplateSel<parquet::test::TestDeltaByteArrayEncoding_BasicRoundTrip_Test>, testing::internal::Types<parquet::PhysicalType<parquet::Type::BYTE_ARRAY>, parquet::PhysicalType<parquet::Type::FIXED_LEN_BYTE_ARRAY>>>::Register' requested here
   TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
   ^
   /build/cpp/googletest_ep-prefix/include/gtest/gtest-typed-test.h:212:27: note: expanded from macro 'TYPED_TEST'
                 CaseName)>::Register("",                                        \
                             ^
   2 errors generated.
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1257250154


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,14 +3058,237 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+  static constexpr std::string_view kEmpty = "";
+
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(descr, pool),

Review Comment:
   Ok, setting `suffix_encoder_(descr, pool)` and others to `..._encoder(nullptr, pool)`. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1268152431


##########
cpp/src/parquet/test_util.cc:
##########
@@ -132,5 +132,52 @@ void random_byte_array(int n, uint32_t seed, uint8_t* buf, ByteArray* out, int m
   random_byte_array(n, seed, buf, out, 0, max_size);
 }
 
+void prefixed_random_byte_array(int n, uint32_t seed, uint8_t* buf, ByteArray* out,
+                                int min_size, int max_size, double prefixed_probability) {
+  std::default_random_engine gen(seed);
+  std::uniform_int_distribution<int> d1(min_size, max_size);
+  std::uniform_int_distribution<int> d2(0, 255);
+  std::uniform_real_distribution<double> d3(0, 1);

Review Comment:
   Can you give those meaningful names?
   ```suggestion
     std::uniform_int_distribution<int> dist_size(min_size, max_size);
     std::uniform_int_distribution<int> dist_byte(0, 255);
     std::uniform_real_distribution<double> dist_has_prefix(0, 1);
   ```



##########
cpp/src/parquet/test_util.h:
##########
@@ -783,8 +789,39 @@ inline void GenerateData<ByteArray>(int num_values, ByteArray* out,
   random_byte_array(num_values, 0, heap->data(), out, 2, max_byte_array_len);
 }
 
+template <typename T>
+inline void GeneratePrefixedData(int num_values, T* out, std::vector<uint8_t>* heap,
+                                 double prefixed_probability) {
+  // seed the prng so failure is deterministic

Review Comment:
   You don't have to define the generic version since there's a specialized one below. Just declare it:
   ```c++
   
   template <typename T>
   void GeneratePrefixedData(int num_values, T* out, std::vector<uint8_t>* heap,
                             double prefixed_probability);
   ```



##########
cpp/src/parquet/test_util.cc:
##########
@@ -132,5 +132,52 @@ void random_byte_array(int n, uint32_t seed, uint8_t* buf, ByteArray* out, int m
   random_byte_array(n, seed, buf, out, 0, max_size);
 }
 
+void prefixed_random_byte_array(int n, uint32_t seed, uint8_t* buf, ByteArray* out,
+                                int min_size, int max_size, double prefixed_probability) {
+  std::default_random_engine gen(seed);
+  std::uniform_int_distribution<int> d1(min_size, max_size);
+  std::uniform_int_distribution<int> d2(0, 255);
+  std::uniform_real_distribution<double> d3(0, 1);
+
+  for (int i = 0; i < n; ++i) {
+    int len = d1(gen);
+    out[i].len = len;
+    out[i].ptr = buf;
+
+    bool do_prefix = d3(gen) < prefixed_probability && i > 0;
+    std::uniform_int_distribution<int> d4(min_size, len);

Review Comment:
   Instead of creating a new distribution in each pass, you could generate a fraction of the length:
   ```c++
     std::uniform_real_distribution<double> dist_prefix_length(0, 1);
     ...
       bool do_prefix = dist_has_prefix(gen) && i > 0;
       int prefix_len = do_prefix ? static_cast<int>(std::ceil(len * dist_prefix_length(gen))) : 0;
   ```
   



##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,43 +1240,62 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
-  explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
-    this->out = out;
-    this->builder = out->builder.get();
-    this->chunk_space_remaining =
-        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+template <typename DType, typename Enable = void>
+struct ArrowBinaryHelper;
+
+template <typename DType>
+struct ArrowBinaryHelper<DType, std::enable_if_t<std::is_same_v<DType, ByteArrayType> ||
+                                                     std::is_same_v<DType, FLBAType>,
+                                                 void>> {
+  explicit ArrowBinaryHelper(typename EncodingTraits<DType>::Accumulator* acc) {
+    builder = acc->builder.get();
+    chunks = &acc->chunks;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 builder->value_data_length(),
+                                                 &chunk_space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<DType>");
+    }
   }
 
   Status PushChunk() {
     std::shared_ptr<::arrow::Array> result;
     RETURN_NOT_OK(builder->Finish(&result));
-    out->chunks.push_back(result);
+    chunks->push_back(std::move(result));
     chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
     return Status::OK();
   }
 
   bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
 
   void UnsafeAppend(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
     builder->UnsafeAppend(data, length);
   }
 
   void UnsafeAppendNull() { builder->UnsafeAppendNull(); }
 
-  Status Append(const uint8_t* data, int32_t length) {
-    chunk_space_remaining -= length;
-    return builder->Append(data, length);
-  }
+  virtual Status Append(const uint8_t* data, int32_t length);

Review Comment:
   This does not need to be `virtual` since you're not subclassing this class anywhere, right?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,43 +1240,62 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
-  explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
-    this->out = out;
-    this->builder = out->builder.get();
-    this->chunk_space_remaining =
-        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+template <typename DType, typename Enable = void>
+struct ArrowBinaryHelper;
+
+template <typename DType>
+struct ArrowBinaryHelper<DType, std::enable_if_t<std::is_same_v<DType, ByteArrayType> ||
+                                                     std::is_same_v<DType, FLBAType>,
+                                                 void>> {
+  explicit ArrowBinaryHelper(typename EncodingTraits<DType>::Accumulator* acc) {
+    builder = acc->builder.get();
+    chunks = &acc->chunks;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 builder->value_data_length(),
+                                                 &chunk_space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<DType>");
+    }
   }
 
   Status PushChunk() {
     std::shared_ptr<::arrow::Array> result;
     RETURN_NOT_OK(builder->Finish(&result));
-    out->chunks.push_back(result);
+    chunks->push_back(std::move(result));
     chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
     return Status::OK();
   }
 
   bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
 
   void UnsafeAppend(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));

Review Comment:
   Why is `UnsafeAppend` non-templated but `Append` is templated?
   Also, why does `UnsafeAppend` have a `DCHECK(CanFit(length))` but not `Append`?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),

Review Comment:
   It could be deferred to another PR, though, assuming you open an issue about it. Also, it applies to many encoders, not just this one.



##########
cpp/src/parquet/test_util.cc:
##########
@@ -132,5 +132,52 @@ void random_byte_array(int n, uint32_t seed, uint8_t* buf, ByteArray* out, int m
   random_byte_array(n, seed, buf, out, 0, max_size);
 }
 
+void prefixed_random_byte_array(int n, uint32_t seed, uint8_t* buf, ByteArray* out,
+                                int min_size, int max_size, double prefixed_probability) {
+  std::default_random_engine gen(seed);
+  std::uniform_int_distribution<int> d1(min_size, max_size);
+  std::uniform_int_distribution<int> d2(0, 255);
+  std::uniform_real_distribution<double> d3(0, 1);

Review Comment:
   Also use `https://en.cppreference.com/w/cpp/numeric/random/bernoulli_distribution` for `dist_has_prefix`



##########
cpp/src/parquet/test_util.cc:
##########
@@ -132,5 +132,52 @@ void random_byte_array(int n, uint32_t seed, uint8_t* buf, ByteArray* out, int m
   random_byte_array(n, seed, buf, out, 0, max_size);
 }
 
+void prefixed_random_byte_array(int n, uint32_t seed, uint8_t* buf, ByteArray* out,
+                                int min_size, int max_size, double prefixed_probability) {
+  std::default_random_engine gen(seed);
+  std::uniform_int_distribution<int> d1(min_size, max_size);
+  std::uniform_int_distribution<int> d2(0, 255);
+  std::uniform_real_distribution<double> d3(0, 1);

Review Comment:
   Oh, and we should probably use the compatibility versions from `arrow/testing/uniform_real.h`.



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1908,4 +1907,304 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void InitData(int nvalues, int repeats, double prefixed_probability) {
+    num_values_ = nvalues * repeats;
+    input_bytes_.resize(num_values_ * sizeof(c_type));
+    output_bytes_.resize(num_values_ * sizeof(c_type));
+    draws_ = reinterpret_cast<c_type*>(input_bytes_.data());
+    decode_buf_ = reinterpret_cast<c_type*>(output_bytes_.data());
+    GeneratePrefixedData<c_type>(nvalues, draws_, &data_buffer_, prefixed_probability);
+
+    // add some repeated values
+    for (int j = 1; j < repeats; ++j) {
+      for (int i = 0; i < nvalues; ++i) {
+        draws_[nvalues * j + i] = draws_[i];
+      }
+    }
+  }
+
+  void Execute(int nvalues, int repeats, double prefixed_probability) {
+    InitData(nvalues, repeats, prefixed_probability);
+    CheckRoundtrip();
+  }
+
+  void ExecuteSpaced(int nvalues, int repeats, int64_t valid_bits_offset,

Review Comment:
   It's looking like you're mostly copy-pasting code from another test class here. You could just hard-code `prefixed_probability` to a reasonable value, or arrange to vary it implicitly between calls to `InitData`.



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1908,4 +1907,304 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void InitData(int nvalues, int repeats, double prefixed_probability) {
+    num_values_ = nvalues * repeats;
+    input_bytes_.resize(num_values_ * sizeof(c_type));
+    output_bytes_.resize(num_values_ * sizeof(c_type));
+    draws_ = reinterpret_cast<c_type*>(input_bytes_.data());
+    decode_buf_ = reinterpret_cast<c_type*>(output_bytes_.data());
+    GeneratePrefixedData<c_type>(nvalues, draws_, &data_buffer_, prefixed_probability);
+
+    // add some repeated values
+    for (int j = 1; j < repeats; ++j) {
+      for (int i = 0; i < nvalues; ++i) {
+        draws_[nvalues * j + i] = draws_[i];
+      }
+    }
+  }
+
+  void Execute(int nvalues, int repeats, double prefixed_probability) {
+    InitData(nvalues, repeats, prefixed_probability);
+    CheckRoundtrip();
+  }
+
+  void ExecuteSpaced(int nvalues, int repeats, int64_t valid_bits_offset,
+                     double null_probability, double prefixed_probability) {
+    InitData(nvalues, repeats, prefixed_probability);
+
+    int64_t size = num_values_ + valid_bits_offset;
+    auto rand = ::arrow::random::RandomArrayGenerator(1923);
+    const auto array = rand.UInt8(size, /*min=*/0, /*max=*/100, null_probability);
+    const auto valid_bits = array->null_bitmap_data();
+    if (valid_bits) {
+      CheckRoundtripSpaced(valid_bits, valid_bits_offset);
+    }
+  }
+
+  void CheckRoundtrip() override {

Review Comment:
   Similarly, it looks like this is exactly the same code as the one for testing `DELTA_LENGTH_BYTE_ARRAY`.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),

Review Comment:
   Did you actually address this comment @rok?



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1908,4 +1907,304 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void InitData(int nvalues, int repeats, double prefixed_probability) {
+    num_values_ = nvalues * repeats;
+    input_bytes_.resize(num_values_ * sizeof(c_type));
+    output_bytes_.resize(num_values_ * sizeof(c_type));
+    draws_ = reinterpret_cast<c_type*>(input_bytes_.data());
+    decode_buf_ = reinterpret_cast<c_type*>(output_bytes_.data());
+    GeneratePrefixedData<c_type>(nvalues, draws_, &data_buffer_, prefixed_probability);
+
+    // add some repeated values
+    for (int j = 1; j < repeats; ++j) {
+      for (int i = 0; i < nvalues; ++i) {
+        draws_[nvalues * j + i] = draws_[i];
+      }
+    }
+  }
+
+  void Execute(int nvalues, int repeats, double prefixed_probability) {
+    InitData(nvalues, repeats, prefixed_probability);
+    CheckRoundtrip();
+  }
+
+  void ExecuteSpaced(int nvalues, int repeats, int64_t valid_bits_offset,
+                     double null_probability, double prefixed_probability) {
+    InitData(nvalues, repeats, prefixed_probability);
+
+    int64_t size = num_values_ + valid_bits_offset;
+    auto rand = ::arrow::random::RandomArrayGenerator(1923);
+    const auto array = rand.UInt8(size, /*min=*/0, /*max=*/100, null_probability);
+    const auto valid_bits = array->null_bitmap_data();
+    if (valid_bits) {
+      CheckRoundtripSpaced(valid_bits, valid_bits_offset);
+    }
+  }
+
+  void CheckRoundtrip() override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+  std::vector<uint8_t> input_bytes_;
+  std::vector<uint8_t> output_bytes_;
+};
+
+using TestDeltaByteArrayEncodingTypes = ::testing::Types<ByteArrayType, FLBAType>;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, /*repeats=*/0, /*prefixed_probability=*/0.1));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 5, 0.2));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 1, 0.3));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_probability*/
+      0, 0.4));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.5, 0.5));
+}
+
+template <typename Type>
+class DeltaByteArrayEncodingDirectPut : public TestEncodingBase<Type> {
+ public:
+  std::unique_ptr<TypedEncoder<Type>> encoder =
+      MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+  std::unique_ptr<TypedDecoder<Type>> decoder =
+      MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+
+  void CheckDirectPut(std::shared_ptr<::arrow::Array> array) {
+    ASSERT_NO_THROW(encoder->Put(*array));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(array->length() - array->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<Type>::Accumulator acc;
+    using BuilderType = typename EncodingTraits<Type>::BuilderType;
+    acc.builder = std::make_unique<BuilderType>(array->type(), default_memory_pool());
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(array->length()),
+                                   static_cast<int>(array->null_count()),
+                                   array->null_bitmap_data(), array->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(array->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    ::arrow::AssertArraysEqual(*array, *result);
+  }
+
+  void CheckRoundtripFLBA() {
+    constexpr int64_t kSize = 50;
+    constexpr int kSeed = 42;
+    constexpr int kByteWidth = 4;
+    ::arrow::random::RandomArrayGenerator rag{kSeed};
+    std::shared_ptr<::arrow::Array> values =
+        rag.FixedSizeBinary(/*size=*/0, /*byte_width=*/kByteWidth);
+    CheckDirectPut(values);
+
+    for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+      rag = ::arrow::random::RandomArrayGenerator(seed);
+      values = rag.FixedSizeBinary(kSize + seed, kByteWidth);
+      CheckDirectPut(values);
+    }
+  }
+
+  void CheckRoundtripByteArray() {
+    constexpr int64_t kSize = 500;
+    constexpr int32_t kMinLength = 0;
+    constexpr int32_t kMaxLength = 10;
+    constexpr int32_t kNumUnique = 10;
+    constexpr double kNullProbability = 0.25;
+    constexpr int kSeed = 42;
+    ::arrow::random::RandomArrayGenerator rag{kSeed};
+    std::shared_ptr<::arrow::Array> values = rag.BinaryWithRepeats(
+        /*size=*/1, /*unique=*/1, kMinLength, kMaxLength, kNullProbability);
+    CheckDirectPut(values);
+
+    for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+      rag = ::arrow::random::RandomArrayGenerator(seed);
+      values = rag.BinaryWithRepeats(kSize, kNumUnique, kMinLength, kMaxLength,
+                                     kNullProbability);
+      CheckDirectPut(values);
+    }
+  }
+
+  void CheckRoundtrip() override {
+    using ArrowType = typename EncodingTraits<Type>::ArrowType;
+    using IsFixedSizeBinary = ::arrow::is_fixed_size_binary_type<ArrowType>;
+
+    if constexpr (IsFixedSizeBinary::value) {
+      CheckRoundtripFLBA();
+    } else {
+      CheckRoundtripByteArray();
+    }
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+TYPED_TEST_SUITE(DeltaByteArrayEncodingDirectPut, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(DeltaByteArrayEncodingDirectPut, DirectPut) {
+  ASSERT_NO_FATAL_FAILURE(this->CheckRoundtrip());
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
+  auto CheckEncode = [](const std::shared_ptr<::arrow::Array>& values,
+                        const std::shared_ptr<Buffer>& encoded) {
+    auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+    ASSERT_TRUE(encoded->Equals(*buf));
+  };
+
+  auto ArrayToInt32Vector = [](const std::shared_ptr<::arrow::Array>& lengths) {
+    std::vector<int32_t> arrays;

Review Comment:
   Why the name?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3064,269 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {

Review Comment:
   The change has not been made apparently.



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1908,4 +1907,304 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void InitData(int nvalues, int repeats, double prefixed_probability) {
+    num_values_ = nvalues * repeats;
+    input_bytes_.resize(num_values_ * sizeof(c_type));
+    output_bytes_.resize(num_values_ * sizeof(c_type));
+    draws_ = reinterpret_cast<c_type*>(input_bytes_.data());
+    decode_buf_ = reinterpret_cast<c_type*>(output_bytes_.data());
+    GeneratePrefixedData<c_type>(nvalues, draws_, &data_buffer_, prefixed_probability);
+
+    // add some repeated values
+    for (int j = 1; j < repeats; ++j) {
+      for (int i = 0; i < nvalues; ++i) {
+        draws_[nvalues * j + i] = draws_[i];
+      }
+    }
+  }
+
+  void Execute(int nvalues, int repeats, double prefixed_probability) {
+    InitData(nvalues, repeats, prefixed_probability);
+    CheckRoundtrip();
+  }
+
+  void ExecuteSpaced(int nvalues, int repeats, int64_t valid_bits_offset,
+                     double null_probability, double prefixed_probability) {
+    InitData(nvalues, repeats, prefixed_probability);
+
+    int64_t size = num_values_ + valid_bits_offset;
+    auto rand = ::arrow::random::RandomArrayGenerator(1923);
+    const auto array = rand.UInt8(size, /*min=*/0, /*max=*/100, null_probability);
+    const auto valid_bits = array->null_bitmap_data();
+    if (valid_bits) {
+      CheckRoundtripSpaced(valid_bits, valid_bits_offset);
+    }
+  }
+
+  void CheckRoundtrip() override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+  std::vector<uint8_t> input_bytes_;
+  std::vector<uint8_t> output_bytes_;
+};
+
+using TestDeltaByteArrayEncodingTypes = ::testing::Types<ByteArrayType, FLBAType>;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, /*repeats=*/0, /*prefixed_probability=*/0.1));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 5, 0.2));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 1, 0.3));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_probability*/
+      0, 0.4));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.5, 0.5));
+}
+
+template <typename Type>
+class DeltaByteArrayEncodingDirectPut : public TestEncodingBase<Type> {
+ public:
+  std::unique_ptr<TypedEncoder<Type>> encoder =
+      MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+  std::unique_ptr<TypedDecoder<Type>> decoder =
+      MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+
+  void CheckDirectPut(std::shared_ptr<::arrow::Array> array) {
+    ASSERT_NO_THROW(encoder->Put(*array));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(array->length() - array->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<Type>::Accumulator acc;
+    using BuilderType = typename EncodingTraits<Type>::BuilderType;
+    acc.builder = std::make_unique<BuilderType>(array->type(), default_memory_pool());
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(array->length()),
+                                   static_cast<int>(array->null_count()),
+                                   array->null_bitmap_data(), array->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(array->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    ::arrow::AssertArraysEqual(*array, *result);
+  }
+
+  void CheckRoundtripFLBA() {
+    constexpr int64_t kSize = 50;
+    constexpr int kSeed = 42;
+    constexpr int kByteWidth = 4;
+    ::arrow::random::RandomArrayGenerator rag{kSeed};
+    std::shared_ptr<::arrow::Array> values =
+        rag.FixedSizeBinary(/*size=*/0, /*byte_width=*/kByteWidth);
+    CheckDirectPut(values);
+
+    for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+      rag = ::arrow::random::RandomArrayGenerator(seed);
+      values = rag.FixedSizeBinary(kSize + seed, kByteWidth);
+      CheckDirectPut(values);
+    }

Review Comment:
   You don't have to recreate a new `RandomArrayGenerator` every time. Just reuse the one created above. It will produce different results everytime.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3058,237 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+  static constexpr std::string_view kEmpty = "";
+
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        // Prefix lengths are encoded using DeltaBitPackEncoder that can be left
+        // uninitialized.
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(descr, pool),
+        last_value_(""),
+        empty_(static_cast<uint32_t>(kEmpty.size()),

Review Comment:
   Idea: if you make the `ByteArray` constructors `constexpr`, you might be able to write:
   ```c++
   static constexpr ByteArray kEmpty = std::string_view("");
   ```
   



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,230 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        auto view = visitor[i + j];
+        len = visitor.len(i + j);
+
+        uint32_t k = 0;
+        const uint32_t common_length =
+            std::min(len, static_cast<uint32_t>(last_value_view.length()));
+        while (k < common_length) {
+          if (last_value_view[k] != view[k]) {
+            break;
+          }
+          k++;
+        }
+
+        last_value_view = view;
+        prefix_lengths[j] = k;
+        const uint32_t suffix_length = len - k;
+        const uint8_t* suffix_ptr = src[i + j].ptr + k;
+
+        // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+        const ByteArray suffix(suffix_length, suffix_ptr);
+        suffixes[j] = suffix;
+      }
+      suffix_encoder_.Put(suffixes.data(), batch_size);
+      prefix_length_encoder_.Put(prefix_lengths.data(), batch_size);
+    }
+    last_value_ = last_value_view;
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.length());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t len = src.len;
+          const uint32_t common_length = std::min(previous_len, len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+struct ByteArrayVisitor {
+  const ByteArray* src;
+  const uint32_t type_length;

Review Comment:
   Or, you know, instead of having:
   ```c++
     template <typename VisitorType>
     void PutInternal(const T* src, int num_values) {
       ...
       auto visitor = VisitorType{src, flba_len};
   ```
   just pass the visitor instance so that you don't have to match the constructor signatures:
   ```c++
     template <typename VisitorType>
     void PutInternal(const T* src, int num_values, VisitorType* visitor) {
   ```
   



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -874,7 +874,7 @@ std::shared_ptr<::arrow::Array> EncodingAdHocTyped<FLBAType>::GetValues(int seed
 }
 
 using EncodingAdHocTypedCases =
-    ::testing::Types<BooleanType, Int32Type, Int64Type, FloatType, DoubleType, FLBAType>;
+    ::testing::Types<BooleanType, Int32Type, Int64Type, FloatType, DoubleType>;

Review Comment:
   Hmm... does this mean FLBA handling is broken for some existing encodings? It's not clear what the consequences are here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1645733698

   @pitrou thanks for the review, I think I addressed all the points, could you take another look?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer [arrow]

Posted by "jorisvandenbossche (via GitHub)" <gi...@apache.org>.
jorisvandenbossche commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1386828655


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1335,13 +1420,7 @@ class PlainByteArrayDecoder : public PlainDecoder<ByteArrayType>,
           if (ARROW_PREDICT_FALSE(len_ < increment)) {
             ParquetException::EofException();
           }
-          if (ARROW_PREDICT_FALSE(!helper.CanFit(value_len))) {
-            // This element would exceed the capacity of a chunk
-            RETURN_NOT_OK(helper.PushChunk());
-            RETURN_NOT_OK(helper.builder->Reserve(num_values - i));
-            RETURN_NOT_OK(helper.builder->ReserveData(
-                std::min<int64_t>(len_, helper.chunk_space_remaining)));

Review Comment:
   See https://github.com/apache/arrow/pull/38437 for context, where this code is being added back partially



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer [arrow]

Posted by "jorisvandenbossche (via GitHub)" <gi...@apache.org>.
jorisvandenbossche commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1386828655


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1335,13 +1420,7 @@ class PlainByteArrayDecoder : public PlainDecoder<ByteArrayType>,
           if (ARROW_PREDICT_FALSE(len_ < increment)) {
             ParquetException::EofException();
           }
-          if (ARROW_PREDICT_FALSE(!helper.CanFit(value_len))) {
-            // This element would exceed the capacity of a chunk
-            RETURN_NOT_OK(helper.PushChunk());
-            RETURN_NOT_OK(helper.builder->Reserve(num_values - i));
-            RETURN_NOT_OK(helper.builder->ReserveData(
-                std::min<int64_t>(len_, helper.chunk_space_remaining)));

Review Comment:
   See https://github.com/apache/arrow/pull/38437 for context



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1157449310


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          const uint8_t* suffix_ptr = src.ptr + j;
+          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+          last_value_view = view;
+          // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  int i = 0;
+  while (i < num_values) {

Review Comment:
   Good point. Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1157766484


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3196,6 +3397,30 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) {
+    return GetInternal(buffer, max_values);
+  }
+  int Decode(FixedLenByteArray* buffer, int max_values) override {

Review Comment:
   Should be solved now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1161828432


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3083,242 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          buffer_, ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  std::unique_ptr<::arrow::Buffer> buffer_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw Status::Invalid("Put not implemented for " + this->descr_->ToString());
+}
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i++) {
+    // Convert to ByteArray, so we can pass to the suffix_encoder_.
+    const ByteArray value = src[i];
+    if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+      throw Status::Invalid("Parquet cannot store strings with size 2GB or more");

Review Comment:
   seems that throw should throw an `ParquetException`, not an status?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3083,242 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          buffer_, ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));

Review Comment:
   why do we make `buffer_` as a member in `DeltaByteArrayEncoder`? It may consume some memory



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3196,6 +3469,43 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) {
+    return GetInternal(buffer, max_values);
+  }
+  int Decode(FixedLenByteArray* buffer, int max_values) override {
+    int decoded_values_size = max_values;
+    if (MultiplyWithOverflow(decoded_values_size,
+                             descr_->type_length() * sizeof(ByteArray),
+                             &decoded_values_size)) {
+      throw ParquetException("excess expansion in DELTA_LENGTH_BYTE_ARRAY");
+    }
+    std::vector<uint8_t> decode_values(decoded_values_size);

Review Comment:
   Should we allocate it from `pool_` to ensure that it's aligned?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3083,242 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          buffer_, ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  std::unique_ptr<::arrow::Buffer> buffer_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw Status::Invalid("Put not implemented for " + this->descr_->ToString());

Review Comment:
   throw Should throw ParquetException.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1159707891


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          const uint8_t* suffix_ptr = src.ptr + j;
+          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+          last_value_view = view;
+          // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);

Review Comment:
   `PutBinaryArray` now passes, but could perhaps be optimized:
   ```cpp
   if (suffix_length == 0) {
     const auto suffix_ptr = reinterpret_cast<const uint8_t*>("");
     const ByteArray suffix(suffix_length, suffix_ptr);
     suffix_encoder_.Put(&suffix, 1);
     return Status::OK();
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1198509202


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,229 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        auto view = visitor[i + j];
+        len = visitor.len(i + j);
+
+        uint32_t k = 0;
+        const uint32_t common_length =
+            std::min(len, static_cast<uint32_t>(last_value_view.length()));
+        while (k < common_length) {
+          if (last_value_view[k] != view[k]) {
+            break;
+          }
+          k++;
+        }
+
+        last_value_view = view;
+        prefix_lengths[j] = k;
+        const auto suffix_length = len - k;
+        const uint8_t* suffix_ptr = src[i + j].ptr + k;
+
+        // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+        const ByteArray suffix(suffix_length, suffix_ptr);
+        suffixes[j] = suffix;
+      }
+      suffix_encoder_.Put(suffixes.data(), batch_size);
+      prefix_length_encoder_.Put(prefix_lengths.data(), batch_size);
+    }
+    last_value_ = last_value_view;
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+struct ByteArrayVisitor {
+  const ByteArray* src;
+  const uint32_t length;
+
+  std::string_view operator[](int i) const {
+    if (ARROW_PREDICT_FALSE(src[i].len >= kMaxByteArraySize)) {
+      throw ParquetException("Parquet cannot store strings with size 2GB or more");
+    }
+    return std::string_view{src[i]};
+  }
+
+  const uint32_t len(int i) const { return src[i].len; }
+};
+
+struct FLBAVisitor {
+  const FLBA* src;
+  const uint32_t length;
+
+  std::string_view operator[](int i) const {
+    return std::string_view{reinterpret_cast<const char*>(src[i].ptr)};

Review Comment:
   You should explicit set the length for flba, otherwise I think it will be regard as C style string and truncate the content after `\0`
   
   ```c++
     std::string_view operator[](int i) const {
       return std::string_view{reinterpret_cast<const char*>(src[i].ptr, flba.length)};
     }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1143217611


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3265,6 +3420,14 @@ std::unique_ptr<Encoder> MakeEncoder(Type::type type_num, Encoding::type encodin
       default:
         throw ParquetException("DELTA_LENGTH_BYTE_ARRAY only supports BYTE_ARRAY");
     }
+  } else if (encoding == Encoding::DELTA_BYTE_ARRAY) {
+    switch (type_num) {
+      case Type::BYTE_ARRAY:
+        return std::make_unique<DeltaByteArrayEncoder<ByteArrayType>>(descr, pool);
+      default:
+        throw ParquetException("DELTA_BYTE_ARRAY only supports BYTE_ARRAY");

Review Comment:
   Great work, though it seems that we can change the error message here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1146206091


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1268,6 +1268,35 @@ struct ArrowBinaryHelper {
   int64_t chunk_space_remaining;
 };
 
+struct ArrowFLBAHelper {
+  explicit ArrowFLBAHelper(::arrow::FixedSizeBinaryBuilder* builder) {
+    this->builder = builder;
+    this->chunk_space_remaining =
+        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+  }
+
+  Status PushChunk() {
+    std::shared_ptr<::arrow::Array> result;
+    RETURN_NOT_OK(builder->Finish(&result));
+    chunks.push_back(result);
+    chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
+    return Status::OK();
+  }
+
+  bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
+
+  Status Append(const uint8_t* data, int32_t length) {
+    chunk_space_remaining -= length;

Review Comment:
   Added.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1142916191


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1268,6 +1268,35 @@ struct ArrowBinaryHelper {
   int64_t chunk_space_remaining;
 };
 
+struct ArrowFLBAHelper {
+  explicit ArrowFLBAHelper(::arrow::FixedSizeBinaryBuilder* builder) {
+    this->builder = builder;
+    this->chunk_space_remaining =
+        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+  }
+
+  Status PushChunk() {
+    std::shared_ptr<::arrow::Array> result;
+    RETURN_NOT_OK(builder->Finish(&result));
+    chunks.push_back(result);
+    chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
+    return Status::OK();
+  }
+
+  bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
+
+  Status Append(const uint8_t* data, int32_t length) {
+    chunk_space_remaining -= length;

Review Comment:
   `DCHECK(CanFit(..))` ?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -2900,8 +2929,222 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+ public:
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr, MemoryPool* pool)
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY,
+                    pool = ::arrow::default_memory_pool()),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+
+    // TODO(rok): optimize using ArrowPoolVector<int32_t> prefix_lengths(num_values);
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          uint32_t previous_len = 0;
+          const ByteArray src{view};

Review Comment:
   Just curious that why FLBA cast to a ByteArray here, wouldn't `T == FLBA` and cast failed?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -2900,8 +2929,222 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+ public:
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr, MemoryPool* pool)
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY,
+                    pool = ::arrow::default_memory_pool()),

Review Comment:
   ```
     explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr, MemoryPool* pool= ::arrow::default_memory_pool())
         : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool)),
   ```



##########
cpp/src/parquet/encoding.cc:
##########
@@ -2900,8 +2929,222 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+ public:
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr, MemoryPool* pool)
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY,
+                    pool = ::arrow::default_memory_pool()),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+
+    // TODO(rok): optimize using ArrowPoolVector<int32_t> prefix_lengths(num_values);
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          uint32_t previous_len = 0;
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+          } else {
+            uint32_t j = 0;
+            while (j < std::min(previous_len, byte_width)) {

Review Comment:
   Can we extract `std::min(previous_len, byte_width)` before loop?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -2900,8 +2929,222 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+ public:
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr, MemoryPool* pool)
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY,
+                    pool = ::arrow::default_memory_pool()),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+
+    // TODO(rok): optimize using ArrowPoolVector<int32_t> prefix_lengths(num_values);
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          uint32_t previous_len = 0;

Review Comment:
   Would it miss the previous len, sinces it is always len.
   
   eg:
   
   ```
   while (j < std::min(previous_len, byte_width)) {}
   ```
   
   would be 
   
   ```
   while (j < 0)
   ```
   
   ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] wjones127 commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "wjones127 (via GitHub)" <gi...@apache.org>.
wjones127 commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1153816452


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          const uint8_t* suffix_ptr = src.ptr + j;
+          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+          last_value_view = view;
+          // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  int i = 0;
+  while (i < num_values) {
+    // Convert to ByteArray so we can pass to the suffix_encoder_.
+    auto value = reinterpret_cast<const ByteArray*>(&src[i]);
+    if (ARROW_PREDICT_FALSE(value->len >= kMaxByteArraySize)) {
+      throw Status::Invalid("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value->ptr), value->len};
+    uint32_t j = 0;
+    while (j < std::min(value->len, static_cast<uint32_t>(last_value_view.length()))) {

Review Comment:
   Maybe the compiler is smarter enough to optimize this, but I wonder if we should pull out this cast and comparison from the loop condition?
   
   There's a similar case above.
   
   ```suggestion
       int32_t common_length = std::min(value->len, static_cast<uint32_t>(last_value_view.length()));
       while (j < common_length) {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] wjones127 commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "wjones127 (via GitHub)" <gi...@apache.org>.
wjones127 commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1154667187


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,182 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {

Review Comment:
   I get a segfault here (`EXC_BAD_ACCESS`) when I run this Python script:
   
   ```python
   import pyarrow as pa
   import pyarrow.parquet as pq
   
   arr_flba = pa.array(
           [str(x).zfill(10) for x in range(100)], type=pa.binary(10))
   
   mixed_table = pa.table({'a': arr_flba})
   
   pq.write_table(mixed_table, 'mixed_table.parquet', compression='none',
                  use_dictionary=False,
                  column_encoding={'a': "DELTA_BYTE_ARRAY"})
   ```
   
   Could you look into that?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1257303698


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1910,5 +1909,310 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void InitData(int nvalues, double null_probability) {
+    const int seed = 42;
+    auto rand = ::arrow::random::RandomArrayGenerator(seed);
+    const int min_prefix_length = 0;
+    const int max_prefix_length = 100;
+    const int max_element_length = 1000;
+    const double prefixed_probability = 0.5;
+
+    const auto prefix_array = std::dynamic_pointer_cast<::arrow::StringArray>(
+        rand.String(nvalues, min_prefix_length, max_prefix_length, null_probability));
+    const auto do_prefix = std::dynamic_pointer_cast<::arrow::BooleanArray>(
+        rand.Boolean(nvalues,
+                     /*true_probability=*/prefixed_probability,
+                     /*null_probability=*/0.0));
+    ::arrow::StringBuilder builder(::arrow::default_memory_pool());
+
+    std::string prefix = "";
+    for (int i = 0; i < nvalues; i++) {
+      if (prefix_array->IsNull(i)) {
+        PARQUET_THROW_NOT_OK(builder.AppendNull());
+      } else {
+        const std::string element = prefix_array->GetString(i);
+        if (do_prefix->Value(i) && prefix.length() < max_element_length) {
+          prefix = prefix.append(element);
+        } else {
+          prefix = element;
+        }
+        PARQUET_THROW_NOT_OK(builder.Append(prefix));
+      }
+    }
+
+    std::shared_ptr<::arrow::StringArray> array;
+    ASSERT_OK(builder.Finish(&array));
+    num_values_ = static_cast<int>(array->length() - array->null_count());
+    draws_ = reinterpret_cast<c_type*>(array->value_data()->mutable_data());
+  }
+
+  void Execute(int nvalues, double null_probability) {
+    InitData(nvalues, null_probability);
+    CheckRoundtrip();
+  }
+
+  void ExecuteSpaced(int nvalues, int repeats, int64_t valid_bits_offset,

Review Comment:
   Indeed. Will fix.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1256735213


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,230 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        auto view = visitor[i + j];
+        len = visitor.len(i + j);
+
+        uint32_t k = 0;
+        const uint32_t common_length =
+            std::min(len, static_cast<uint32_t>(last_value_view.length()));
+        while (k < common_length) {
+          if (last_value_view[k] != view[k]) {
+            break;
+          }
+          k++;
+        }
+
+        last_value_view = view;
+        prefix_lengths[j] = k;
+        const uint32_t suffix_length = len - k;
+        const uint8_t* suffix_ptr = src[i + j].ptr + k;
+
+        // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+        const ByteArray suffix(suffix_length, suffix_ptr);
+        suffixes[j] = suffix;
+      }
+      suffix_encoder_.Put(suffixes.data(), batch_size);
+      prefix_length_encoder_.Put(prefix_lengths.data(), batch_size);
+    }
+    last_value_ = last_value_view;
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.length());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t len = src.len;
+          const uint32_t common_length = std::min(previous_len, len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+struct ByteArrayVisitor {
+  const ByteArray* src;
+  const uint32_t type_length;

Review Comment:
   Added a comment. We don't seem to use `[[maybe_unused]` elsewhere at the moment.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1627396079

   Just make sure that code is ok and can be reviewed. This patch is blocked for a long time, and if problem finished, we can check in just after 13.0 is released. Just make sure the encoding is working well :-)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1623916665

   Well, is the PR ready? @rok 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1261894605


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -874,7 +874,7 @@ std::shared_ptr<::arrow::Array> EncodingAdHocTyped<FLBAType>::GetValues(int seed
 }
 
 using EncodingAdHocTypedCases =
-    ::testing::Types<BooleanType, Int32Type, Int64Type, FloatType, DoubleType, FLBAType>;
+    ::testing::Types<BooleanType, Int32Type, Int64Type, FloatType, DoubleType>;

Review Comment:
   `FLBAType` was removed due to changes to `EncodingTraits<FLBAType>` above. Perhaps we could do something like:
   ```
     if constexpr (IsFixedSizeBinary::value) {
       using BuilderType = typename EncodingTraits<ParquetType>::BuilderType;
     } else {
       using BuilderType = typename EncodingTraits<ParquetType>::Accumulator;
     }
   ```
   to get arround this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] jorisvandenbossche commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "jorisvandenbossche (via GitHub)" <gi...@apache.org>.
jorisvandenbossche commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1687879350

   We should probably update the python docstring as well:
   
   https://github.com/apache/arrow/blob/fe750ed10531c47131b447397e67486656cf8135/python/pyarrow/parquet/core.py#L822-L827
   
   (which was clearly already outdated before this PR as well! ;))


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1682973018

   Thanks for the review @pitrou!
   I think I addressed everything, could you please do another pass?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1295885665


##########
cpp/src/parquet/test_util.h:
##########
@@ -783,8 +789,33 @@ inline void GenerateData<ByteArray>(int num_values, ByteArray* out,
   random_byte_array(num_values, 0, heap->data(), out, 2, max_byte_array_len);
 }
 
+template <typename T>
+inline void GeneratePrefixedData(int num_values, T* out, std::vector<uint8_t>* heap,
+                                 double prefixed_probability);
+
+template <>
+inline void GeneratePrefixedData(int num_values, ByteArray* out,
+                                 std::vector<uint8_t>* heap,
+                                 double prefixed_probability) {
+  // seed the prng so failure is deterministic
+  int max_byte_array_len = 12;
+  heap->resize(num_values * max_byte_array_len);

Review Comment:
   The comment would deserve being put just above the line it applies to :-)
   ```suggestion
     int max_byte_array_len = 12;
     heap->resize(num_values * max_byte_array_len);
     // seed the prng so failure is deterministic
   ```



##########
cpp/src/parquet/test_util.h:
##########
@@ -783,8 +789,33 @@ inline void GenerateData<ByteArray>(int num_values, ByteArray* out,
   random_byte_array(num_values, 0, heap->data(), out, 2, max_byte_array_len);
 }
 
+template <typename T>
+inline void GeneratePrefixedData(int num_values, T* out, std::vector<uint8_t>* heap,
+                                 double prefixed_probability);
+
+template <>
+inline void GeneratePrefixedData(int num_values, ByteArray* out,
+                                 std::vector<uint8_t>* heap,
+                                 double prefixed_probability) {
+  // seed the prng so failure is deterministic
+  int max_byte_array_len = 12;
+  heap->resize(num_values * max_byte_array_len);
+  prefixed_random_byte_array(num_values, /*seed=*/0, heap->data(), out, /*min_size=*/2,
+                             /*max_size=*/max_byte_array_len, prefixed_probability);
+}
+
 static constexpr int kGenerateDataFLBALength = 8;
 
+template <>
+inline void GeneratePrefixedData<FLBA>(int num_values, FLBA* out,
+                                       std::vector<uint8_t>* heap,
+                                       double prefixed_probability) {
+  // seed the prng so failure is deterministic

Review Comment:
   Same here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1295920278


##########
cpp/src/parquet/test_util.cc:
##########
@@ -132,5 +133,54 @@ void random_byte_array(int n, uint32_t seed, uint8_t* buf, ByteArray* out, int m
   random_byte_array(n, seed, buf, out, 0, max_size);
 }
 
+void prefixed_random_byte_array(int n, uint32_t seed, uint8_t* buf, ByteArray* out,
+                                int min_size, int max_size, double prefixed_probability) {
+  std::default_random_engine gen(seed);
+  std::uniform_int_distribution<int> dist_size(min_size, max_size);
+  std::uniform_int_distribution<int> dist_byte(0, 255);
+  std::bernoulli_distribution dist_has_prefix(prefixed_probability);
+  std::uniform_real_distribution<double> dist_prefix_length(0, 1);
+
+  for (int i = 0; i < n; ++i) {
+    int len = dist_size(gen);
+    out[i].len = len;
+    out[i].ptr = buf;
+
+    bool do_prefix = dist_has_prefix(gen) && i > 0;
+    std::uniform_int_distribution<int> d4(min_size, len);
+    int prefix_len =
+        do_prefix ? static_cast<int>(std::ceil(len * dist_prefix_length(gen))) : 0;

Review Comment:
   Hmm, you should also limit yourself to the length of the previous entry. Something like:
   ```suggestion
       int prefix_len = 0;
       if (do_prefix) {
         int max_prefix_len = std::min(len, out[i - 1].len);
         prefix_len = static_cast<int>(std::ceil(max_prefix_len * dist_prefix_length(gen)));
       }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1299812029


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1908,4 +1909,249 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestDeltaLengthByteArrayEncoding<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+  static constexpr double prefixed_probability = 0.5;
+
+  void InitData(int nvalues, int repeats) {
+    num_values_ = nvalues * repeats;
+    input_bytes_.resize(num_values_ * sizeof(c_type));
+    output_bytes_.resize(num_values_ * sizeof(c_type));
+    draws_ = reinterpret_cast<c_type*>(input_bytes_.data());
+    decode_buf_ = reinterpret_cast<c_type*>(output_bytes_.data());
+    GeneratePrefixedData<c_type>(nvalues, draws_, &data_buffer_, prefixed_probability);
+
+    // add some repeated values
+    for (int j = 1; j < repeats; ++j) {
+      for (int i = 0; i < nvalues; ++i) {
+        draws_[nvalues * j + i] = draws_[i];
+      }
+    }
+  }
+
+  Encoding::type GetEncoding() override { return Encoding::DELTA_BYTE_ARRAY; }
+
+ protected:
+  USING_BASE_MEMBERS();
+  std::vector<uint8_t> input_bytes_;
+  std::vector<uint8_t> output_bytes_;
+};
+
+using TestDeltaByteArrayEncodingTypes = ::testing::Types<ByteArrayType, FLBAType>;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, /*repeats=*/0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 5));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 1));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_probability*/
+      0));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.5));
+}
+
+template <typename Type>
+class DeltaByteArrayEncodingDirectPut : public TestEncodingBase<Type> {
+ public:
+  std::unique_ptr<TypedEncoder<Type>> encoder =
+      MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+  std::unique_ptr<TypedDecoder<Type>> decoder =
+      MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+
+  void CheckDirectPut(std::shared_ptr<::arrow::Array> array) {
+    ASSERT_NO_THROW(encoder->Put(*array));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(array->length() - array->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<Type>::Accumulator acc;
+    using BuilderType = typename EncodingTraits<Type>::BuilderType;
+    acc.builder = std::make_unique<BuilderType>(array->type(), default_memory_pool());
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(array->length()),
+                                   static_cast<int>(array->null_count()),
+                                   array->null_bitmap_data(), array->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(array->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    ::arrow::AssertArraysEqual(*array, *result);
+  }
+
+  void CheckRoundtripFLBA() {
+    constexpr int64_t kSize = 50;
+    constexpr int kSeed = 42;
+    constexpr int kByteWidth = 4;
+    ::arrow::random::RandomArrayGenerator rag{kSeed};
+    std::shared_ptr<::arrow::Array> values =
+        rag.FixedSizeBinary(/*size=*/0, /*byte_width=*/kByteWidth);
+    CheckDirectPut(values);
+
+    for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+      values = rag.FixedSizeBinary(kSize + seed, kByteWidth);
+      CheckDirectPut(values);
+    }
+  }
+
+  void CheckRoundtripByteArray() {
+    constexpr int64_t kSize = 500;
+    constexpr int32_t kMinLength = 0;
+    constexpr int32_t kMaxLength = 10;
+    constexpr int32_t kNumUnique = 10;
+    constexpr double kNullProbability = 0.25;
+    constexpr int kSeed = 42;
+    ::arrow::random::RandomArrayGenerator rag{kSeed};
+    std::shared_ptr<::arrow::Array> values = rag.BinaryWithRepeats(
+        /*size=*/1, /*unique=*/1, kMinLength, kMaxLength, kNullProbability);
+    CheckDirectPut(values);
+
+    for (int i = 0; i < 10; ++i) {
+      values = rag.BinaryWithRepeats(kSize, kNumUnique, kMinLength, kMaxLength,
+                                     kNullProbability);
+      CheckDirectPut(values);
+    }
+  }
+
+  void CheckRoundtrip() override {
+    using ArrowType = typename EncodingTraits<Type>::ArrowType;
+    using IsFixedSizeBinary = ::arrow::is_fixed_size_binary_type<ArrowType>;
+
+    if constexpr (IsFixedSizeBinary::value) {
+      CheckRoundtripFLBA();
+    } else {
+      CheckRoundtripByteArray();
+    }
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+TYPED_TEST_SUITE(DeltaByteArrayEncodingDirectPut, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(DeltaByteArrayEncodingDirectPut, DirectPut) {
+  ASSERT_NO_FATAL_FAILURE(this->CheckRoundtrip());
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
+  auto CheckEncode = [](const std::shared_ptr<::arrow::Array>& values,
+                        const std::shared_ptr<Buffer>& encoded) {
+    auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+    ASSERT_TRUE(encoded->Equals(*buf));
+  };
+
+  auto ArrayToInt32Vector = [](const std::shared_ptr<::arrow::Array>& lengths) {
+    std::vector<int32_t> vector;
+    auto data_ptr = checked_cast<::arrow::Int32Array*>(lengths.get());
+    for (int i = 0; i < lengths->length(); ++i) {
+      vector.push_back(data_ptr->GetView(i));
+    }
+    return vector;
+  };
+
+  auto CheckDecode = [](std::shared_ptr<Buffer> buf,
+                        std::shared_ptr<::arrow::Array> values) {
+    int num_values = static_cast<int>(values->length());
+    auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(num_values, result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *upcast_result);
+  };
+
+  auto CheckEncodeDecode =
+      [&](std::string_view values, std::shared_ptr<::arrow::Array> prefix_lengths,
+          std::shared_ptr<::arrow::Array> suffix_lengths, std::string_view suffix_data) {
+        auto encoded =
+            ::arrow::ConcatenateBuffers({DeltaEncode(ArrayToInt32Vector(prefix_lengths)),

Review Comment:
   Yes!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1447608266

   (Can this patch go ahead now?)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1124583494


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3265,6 +3420,14 @@ std::unique_ptr<Encoder> MakeEncoder(Type::type type_num, Encoding::type encodin
       default:
         throw ParquetException("DELTA_LENGTH_BYTE_ARRAY only supports BYTE_ARRAY");
     }
+  } else if (encoding == Encoding::DELTA_BYTE_ARRAY) {
+    switch (type_num) {
+      case Type::BYTE_ARRAY:
+        return std::make_unique<DeltaByteArrayEncoder<ByteArrayType>>(descr, pool);
+      default:
+        throw ParquetException("DELTA_BYTE_ARRAY only supports BYTE_ARRAY");

Review Comment:
   Seems standard says it support FIXED_LENGTH_BYTE_ARRAY. We can add a todo here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1205518569


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,293 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void CheckRoundtrip() override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType, FLBAType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 2));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_prob*/ 0));
+
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 200));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.1));
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPut) {
+  const int64_t size = 50;
+  const int32_t min_length = 0;
+  const int32_t max_length = 10;
+  const int32_t num_unique = 10;
+  const double null_probability = 0.25;
+  auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  ::arrow::random::RandomArrayGenerator rag(42);
+  auto values = rag.String(0, min_length, max_length, null_probability);

Review Comment:
   I would do something like:
   * generate random suffix strings (as a StringArray)
   * generate random prefix lengths (as a Int8Array perhaps)
   * build the final array by combining those two
   
   You also want to optionally have nulls.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1195472664


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,25 +1240,35 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
+template <typename DType>
+struct ArrowBinaryHelper;
+
+template <>
+struct ArrowBinaryHelper<ByteArrayType> {
   explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
     this->out = out;
     this->builder = out->builder.get();
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 this->builder->value_data_length(),
+                                                 &this->chunk_space_remaining))) {

Review Comment:
   `kBinaryMemoryLimit` is `int32_t` while `this->builder->value_data_length()` is `int64_t`, see definitions below. Not sure how how often would the builder grow to `std::numeric_limits<int32_t>::max()` though. I'm happy to switch to regular subtraction if desired.
   
   
   ```cpp
   constexpr int64_t kBinaryMemoryLimit = std::numeric_limits<int32_t>::max() - 1;
   ```
   
   ```cpp
   int64_t value_data_length() const { return byte_builder_.length(); }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1195472664


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,25 +1240,35 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
+template <typename DType>
+struct ArrowBinaryHelper;
+
+template <>
+struct ArrowBinaryHelper<ByteArrayType> {
   explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
     this->out = out;
     this->builder = out->builder.get();
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 this->builder->value_data_length(),
+                                                 &this->chunk_space_remaining))) {

Review Comment:
   `kBinaryMemoryLimit` is `int32_t` while `this->builder->value_data_length()` is `int64_t`, see definitions below. Not sure how often would the builder grow to `std::numeric_limits<int32_t>::max()` though. I'm happy to switch to regular subtraction if desired.
   
   
   ```cpp
   constexpr int64_t kBinaryMemoryLimit = std::numeric_limits<int32_t>::max() - 1;
   ```
   
   ```cpp
   int64_t value_data_length() const { return byte_builder_.length(); }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1197178782


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,293 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void CheckRoundtrip() override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType, FLBAType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));

Review Comment:
   Indeed. How about something like: https://github.com/apache/arrow/pull/14341/commits/66e8695822008582d112e92551c223479a4c6483 - it doesn't have direct control over prefix length distribution, but it's a bit simpler.
   
   cc @mapleFU (thanks for the help!)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1196628756


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,12 +3079,243 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))),
+        prefix_lengths_(kBatchSize_, ::arrow::stl::allocator<int32_t>(pool_)) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+  ArrowPoolVector<int32_t> prefix_lengths_;

Review Comment:
   You can even use a local `std::array<int32_t, kBatchSize> lengths;` as you already did in `DeltaLengthByteArrayEncoder`. I don't know if that would make any difference, though.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1147031185


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3062,226 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+    uint32_t previous_len = byte_width;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = byte_width;
+          } else {
+            uint32_t j = 0;
+            while (j < previous_len) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(byte_width - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_TRUE(view.size() > kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = src.len;
+          } else {
+            uint32_t j = 0;
+            while (j < std::min(previous_len, src.len)) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  string_view last_value_;
+};
+
+template <>
+inline void DeltaByteArrayEncoder<FLBAType>::Put(const FixedLenByteArray* src,
+                                                 int num_values) {
+  if (descr_->type_length() == 0) {
+    return;
+  }
+  for (int i = 0; i < num_values; ++i) {
+    // Write the result to the output stream
+    DCHECK(src[i].ptr != nullptr) << "Value ptr cannot be NULL";
+    PARQUET_THROW_NOT_OK(sink_.Append(src[i].ptr, descr_->type_length()));

Review Comment:
   Indeed! Changed. Still need to add test.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1148411542


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,180 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;

Review Comment:
   `previous_len = last_value_.size();` ?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3196,6 +3397,30 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) {
+    return GetInternal(buffer, max_values);
+  }
+  int Decode(FixedLenByteArray* buffer, int max_values) override {

Review Comment:
   This is a bit confusing, why `DeltaByteArrayFLBADecoder` cannot decode FLBA?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1149884939


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,180 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;

Review Comment:
   With your changes it's now gone.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1157450054


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,285 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  virtual void CheckRoundtrip() {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits, int64_t valid_bits_offset) {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+// TODO: add FLBAType and Decimal type tests
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 2));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_prob*/ 0));
+
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 200));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.1));
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPut) {
+  const int64_t size = 50;
+  const int32_t min_length = 0;
+  const int32_t max_length = 10;
+  const int32_t num_unique = 10;
+  const double null_probability = 0.25;
+  auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  ::arrow::random::RandomArrayGenerator rag(42);
+  auto values = rag.String(0, min_length, max_length, null_probability);
+  CheckSeed(values);
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    rag = ::arrow::random::RandomArrayGenerator(seed);
+
+    values = rag.String(size, min_length, max_length, null_probability);
+    CheckSeed(values);
+
+    values =
+        rag.BinaryWithRepeats(size, num_unique, min_length, max_length, null_probability);
+    CheckSeed(values);
+  }
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPutFixedLength) {
+  const int64_t size = 50;
+  const double null_probability = 0.25;
+  ::arrow::random::RandomArrayGenerator rag(0);
+  auto encoder = MakeTypedEncoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<FLBAType>::Accumulator acc(values->type());
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    for (auto length : {0, 10, 100, 1000}) {
+      rag = ::arrow::random::RandomArrayGenerator(seed);
+      auto values = rag.FixedSizeBinary(size, length, null_probability);
+      CheckSeed(values);
+    }
+  }
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
+  auto CheckEncode = [](std::shared_ptr<::arrow::Array> values,
+                        std::shared_ptr<::arrow::Array> prefix_lengths,
+                        std::shared_ptr<::arrow::Array> suffix_lengths,
+                        std::string_view value) {
+    auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    auto prefix_lengths_encoder =
+        MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
+    ASSERT_NO_THROW(prefix_lengths_encoder->Put(*prefix_lengths));
+    auto prefix_lengths_buf = prefix_lengths_encoder->FlushValues();
+
+    auto encoded_prefix_lengths_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
+
+    auto suffix_lengths_encoder =
+        MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
+    ASSERT_NO_THROW(suffix_lengths_encoder->Put(*suffix_lengths));
+    auto suffix_lengths_buf = suffix_lengths_encoder->FlushValues();
+    auto encoded_values_buf =
+        SliceBuffer(buf, prefix_lengths_buf->size() + suffix_lengths_buf->size());
+
+    auto encoded_prefix_length_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
+    EXPECT_TRUE(prefix_lengths_buf->Equals(*encoded_prefix_length_buf));
+    auto encoded_suffix_length_buf =
+        SliceBuffer(buf, prefix_lengths_buf->size(), suffix_lengths_buf->size());
+    EXPECT_TRUE(suffix_lengths_buf->Equals(*encoded_suffix_length_buf));
+    EXPECT_EQ(value, encoded_values_buf->ToString());
+  };
+
+  auto arrayToI32 = [](const std::shared_ptr<::arrow::Array>& lengths) {
+    std::vector<int32_t> arrays;
+    auto data_ptr = checked_cast<::arrow::Int32Array*>(lengths.get());
+    for (int i = 0; i < lengths->length(); ++i) {
+      arrays.push_back(data_ptr->GetView(i));
+    }
+    return arrays;
+  };
+
+  auto CheckDecode = [](std::shared_ptr<Buffer> buf,
+                        std::shared_ptr<::arrow::Array> values) {
+    int num_values = static_cast<int>(values->length());
+    auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(num_values, result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *upcast_result);
+  };
+
+  auto checkEncodeDecode = [&](std::string_view values,
+                               std::shared_ptr<::arrow::Array> prefix_lengths,
+                               std::shared_ptr<::arrow::Array> suffix_lengths,
+                               std::string_view suffix_data) {
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::utf8(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::large_utf8(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::binary(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::large_binary(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+
+    auto encoded = ::arrow::ConcatenateBuffers({DeltaEncode(arrayToI32(prefix_lengths)),
+                                                DeltaEncode(arrayToI32(suffix_lengths)),
+                                                std::make_shared<Buffer>(suffix_data)})
+                       .ValueOrDie();
+
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::utf8(), values));
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_utf8(), values));
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::binary(), values));
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
+  };
+
+  {
+    auto values = R"(["axis", "axle", "babble", "babyhood"])";

Review Comment:
   I'll push an example soon.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1157748887


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,285 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  virtual void CheckRoundtrip() {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits, int64_t valid_bits_offset) {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+// TODO: add FLBAType and Decimal type tests
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 2));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_prob*/ 0));
+
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 200));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.1));
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPut) {
+  const int64_t size = 50;
+  const int32_t min_length = 0;
+  const int32_t max_length = 10;
+  const int32_t num_unique = 10;
+  const double null_probability = 0.25;
+  auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  ::arrow::random::RandomArrayGenerator rag(42);
+  auto values = rag.String(0, min_length, max_length, null_probability);
+  CheckSeed(values);
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    rag = ::arrow::random::RandomArrayGenerator(seed);
+
+    values = rag.String(size, min_length, max_length, null_probability);
+    CheckSeed(values);
+
+    values =
+        rag.BinaryWithRepeats(size, num_unique, min_length, max_length, null_probability);
+    CheckSeed(values);
+  }
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPutFixedLength) {
+  const int64_t size = 50;
+  const double null_probability = 0.25;
+  ::arrow::random::RandomArrayGenerator rag(0);
+  auto encoder = MakeTypedEncoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<FLBAType>::Accumulator acc(values->type());
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    for (auto length : {0, 10, 100, 1000}) {
+      rag = ::arrow::random::RandomArrayGenerator(seed);
+      auto values = rag.FixedSizeBinary(size, length, null_probability);
+      CheckSeed(values);
+    }
+  }
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
+  auto CheckEncode = [](std::shared_ptr<::arrow::Array> values,
+                        std::shared_ptr<::arrow::Array> prefix_lengths,
+                        std::shared_ptr<::arrow::Array> suffix_lengths,
+                        std::string_view value) {
+    auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    auto prefix_lengths_encoder =
+        MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
+    ASSERT_NO_THROW(prefix_lengths_encoder->Put(*prefix_lengths));
+    auto prefix_lengths_buf = prefix_lengths_encoder->FlushValues();
+
+    auto encoded_prefix_lengths_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
+
+    auto suffix_lengths_encoder =
+        MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
+    ASSERT_NO_THROW(suffix_lengths_encoder->Put(*suffix_lengths));
+    auto suffix_lengths_buf = suffix_lengths_encoder->FlushValues();
+    auto encoded_values_buf =
+        SliceBuffer(buf, prefix_lengths_buf->size() + suffix_lengths_buf->size());
+
+    auto encoded_prefix_length_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
+    EXPECT_TRUE(prefix_lengths_buf->Equals(*encoded_prefix_length_buf));
+    auto encoded_suffix_length_buf =
+        SliceBuffer(buf, prefix_lengths_buf->size(), suffix_lengths_buf->size());
+    EXPECT_TRUE(suffix_lengths_buf->Equals(*encoded_suffix_length_buf));
+    EXPECT_EQ(value, encoded_values_buf->ToString());
+  };
+
+  auto arrayToI32 = [](const std::shared_ptr<::arrow::Array>& lengths) {
+    std::vector<int32_t> arrays;
+    auto data_ptr = checked_cast<::arrow::Int32Array*>(lengths.get());
+    for (int i = 0; i < lengths->length(); ++i) {
+      arrays.push_back(data_ptr->GetView(i));
+    }
+    return arrays;
+  };
+
+  auto CheckDecode = [](std::shared_ptr<Buffer> buf,
+                        std::shared_ptr<::arrow::Array> values) {
+    int num_values = static_cast<int>(values->length());
+    auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(num_values, result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *upcast_result);
+  };
+
+  auto checkEncodeDecode = [&](std::string_view values,
+                               std::shared_ptr<::arrow::Array> prefix_lengths,
+                               std::shared_ptr<::arrow::Array> suffix_lengths,
+                               std::string_view suffix_data) {
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::utf8(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::large_utf8(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::binary(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::large_binary(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+
+    auto encoded = ::arrow::ConcatenateBuffers({DeltaEncode(arrayToI32(prefix_lengths)),
+                                                DeltaEncode(arrayToI32(suffix_lengths)),
+                                                std::make_shared<Buffer>(suffix_data)})
+                       .ValueOrDie();
+
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::utf8(), values));
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_utf8(), values));
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::binary(), values));
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
+  };
+
+  {
+    auto values = R"(["axis", "axle", "babble", "babyhood"])";

Review Comment:
   I'm trying this example but encoded result currently doesn't match up:
   ```cpp
     {
       auto values = R"(["καλημέρα", "καμηλιέρη", "καμηλιέρη", "καλημέρα"])";
       auto prefix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([0, 2, 9, 2])");
       auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([8, 7, 0, 6])");
       constexpr std::string_view suffix_data = "καλημέραμηλιέρηλημέρα";
       CheckEncodeDecode(values, prefix_lengths, suffix_lengths, suffix_data);
     }
   ```
   
   ```
   Value of: prefix_lengths_buf->Equals(*encoded_prefix_length_buf)
     Actual: false
   Expected: true
   
   Value of: suffix_lengths_buf->Equals(*encoded_suffix_length_buf)
     Actual: false
   Expected: true
   
   Expected equality of these values:
     value
       Which is: "\xCE\xBA\xCE\xB1\xCE\xBB\xCE\xB7\xCE\xBC\xCE\xAD\xCF\x81\xCE\xB1\xCE\xBC\xCE\xB7\xCE\xBB\xCE\xB9\xCE\xAD\xCF\x81\xCE\xB7\xCE\xBB\xCE\xB7\xCE\xBC\xCE\xAD\xCF\x81\xCE\xB1"
       As Text: "καλημέραμηλιέρηλημέρα"
     encoded_values_buf->ToString()
       Which is: "\xCE\xBA\xCE\xB1\xCE\xBB\xCE\xB7\xCE\xBC\xCE\xAD\xCF\x81\xCE\xB1\xBC\xCE\xB7\xCE\xBB\xCE\xB9\xCE\xAD\xCF\x81\xCE\xB7\xBB\xCE\xB7\xCE\xBC\xCE\xAD\xCF\x81\xCE\xB1"
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1158125518


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3187,21 +3203,74 @@ void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
 
     auto view = string_view{reinterpret_cast<const char*>(value->ptr), value->len};
     uint32_t j = 0;
-    while (j < std::min(value->len, static_cast<uint32_t>(last_value_view.length()))) {
+    const uint32_t common_length =
+        std::min(value->len, static_cast<uint32_t>(last_value_view.length()));
+    while (j < common_length) {
       if (last_value_view[j] != view[j]) {
         break;
       }
       j++;
     }
 
+    last_value_view = view;
     prefix_lengths[i] = j;
-    const uint8_t* suffix_ptr = value->ptr + j;
-    const uint32_t suffix_length = static_cast<uint32_t>(value->len - j);
+    const auto suffix_length = static_cast<uint32_t>(value->len - j);
+    const uint8_t* suffix_ptr;
+    if (suffix_length == 0) {
+      suffix_ptr = reinterpret_cast<const uint8_t*>("");
+    } else {
+      suffix_ptr = value->ptr + j;
+    }
+    // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+    const ByteArray suffix(suffix_length, suffix_ptr);
+    suffix_encoder_.Put(&suffix, 1);
+  }
+  prefix_length_encoder_.Put(prefix_lengths.data(), num_values);
+  last_value_ = last_value_view;
+}
+
+template <>
+void DeltaByteArrayEncoder<FLBAType>::Put(const FLBA* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+  const int32_t len = descr_->type_length();
+
+  if (ARROW_PREDICT_FALSE(len >= static_cast<int32_t>(kMaxByteArraySize))) {
+    throw Status::Invalid("Parquet cannot store strings with size 2GB or more");
+  }
+
+  for (int i = 0; i < num_values; i++) {
+    // Convert to ByteArray so we can pass to the suffix_encoder_.
+    const FLBA* value = reinterpret_cast<const FLBA*>(&src[i].ptr);
+
+    auto view = string_view{reinterpret_cast<const char*>(value->ptr),
+                            static_cast<uint32_t>(len)};
+    int32_t j = 0;
+    const int32_t common_length =
+        std::min(len, static_cast<int32_t>(last_value_view.length()));
+    while (j < common_length) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
     last_value_view = view;
+    prefix_lengths[i] = j;
+    const auto suffix_length = static_cast<uint32_t>(len - j);
+    const uint8_t* suffix_ptr;
+    if (suffix_length == 0) {
+      suffix_ptr = reinterpret_cast<const uint8_t*>("");

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] wgtmac commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1159204933


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3083,242 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, nullptr)) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          buffer_, ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);

Review Comment:
   ```
   [----------] 3 tests from DeltaByteArrayEncodingAdHoc
   [ RUN      ] DeltaByteArrayEncodingAdHoc.ArrowBinaryDirectPut
   /arrow/cpp/src/arrow/buffer_builder.h:138:27: runtime error: null pointer passed as argument 2, which is declared to never be null
   /usr/include/string.h:44:28: note: nonnull attribute specified here
       #0 0x55a380a6b658 in arrow::BufferBuilder::UnsafeAppend(void const*, long) /arrow/cpp/src/arrow/buffer_builder.h:138:5
       #1 0x7f976e1ca580 in parquet::(anonymous namespace)::DeltaLengthByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::Put(parquet::ByteArray const*, int) /arrow/cpp/src/parquet/encoding.cc:2719:11
       #2 0x7f976e1f7f7f in void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'(std::basic_string_view<char, std::char_traits<char> >)::operator()(std::basic_string_view<char, std::char_traits<char> >) const /arrow/cpp/src/parquet/encoding.cc:3150:5
   ```
   
   The ASAN & UBSAN check fails here, though the line number does not appear to be precise. If `suffix_length` is 0, we probably can skip calling `suffix_encoder_.Put` as there is nothing to concatenate.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1297146185


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3063,241 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+  static constexpr std::string_view kEmpty = "";
+
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(/*descr=*/nullptr, pool),
+        suffix_encoder_(descr, pool),
+        last_value_(""),
+        empty_(static_cast<uint32_t>(kEmpty.size()),
+               reinterpret_cast<const uint8_t*>(kEmpty.data())) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != nullptr) {
+      if (buffer_ == nullptr) {
+        PARQUET_ASSIGN_OR_THROW(buffer_,
+                                ::arrow::AllocateResizableBuffer(num_values * sizeof(T),
+                                                                 this->memory_pool()));
+      } else {
+        PARQUET_THROW_NOT_OK(buffer_->Resize(num_values * sizeof(T), false));
+      }
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t flba_len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, flba_len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        const int idx = i + j;
+        const auto view = visitor[idx];
+        const uint32_t len = static_cast<uint32_t>(view.length());
+
+        uint32_t common_prefix_length = 0;
+        const uint32_t maximum_common_prefix_length =
+            std::min(len, static_cast<uint32_t>(last_value_view.length()));
+        while (common_prefix_length < maximum_common_prefix_length) {
+          if (last_value_view[common_prefix_length] != view[common_prefix_length]) {
+            break;
+          }
+          common_prefix_length++;
+        }
+
+        last_value_view = view;
+        prefix_lengths[j] = common_prefix_length;
+        const uint32_t suffix_length = len - common_prefix_length;
+        const uint8_t* suffix_ptr = src[idx].ptr + common_prefix_length;
+
+        // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+        const ByteArray suffix(suffix_length, suffix_ptr);
+        suffixes[j] = suffix;
+      }
+      suffix_encoder_.Put(suffixes.data(), batch_size);
+      prefix_length_encoder_.Put(prefix_lengths.data(), batch_size);
+    }
+    last_value_ = last_value_view;
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.length());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.

Review Comment:
   Removed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1297745970


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1908,4 +1909,249 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestDeltaLengthByteArrayEncoding<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+  static constexpr double prefixed_probability = 0.5;
+
+  void InitData(int nvalues, int repeats) {
+    num_values_ = nvalues * repeats;
+    input_bytes_.resize(num_values_ * sizeof(c_type));
+    output_bytes_.resize(num_values_ * sizeof(c_type));
+    draws_ = reinterpret_cast<c_type*>(input_bytes_.data());
+    decode_buf_ = reinterpret_cast<c_type*>(output_bytes_.data());
+    GeneratePrefixedData<c_type>(nvalues, draws_, &data_buffer_, prefixed_probability);
+
+    // add some repeated values
+    for (int j = 1; j < repeats; ++j) {
+      for (int i = 0; i < nvalues; ++i) {
+        draws_[nvalues * j + i] = draws_[i];
+      }
+    }
+  }
+
+  Encoding::type GetEncoding() override { return Encoding::DELTA_BYTE_ARRAY; }
+
+ protected:
+  USING_BASE_MEMBERS();
+  std::vector<uint8_t> input_bytes_;
+  std::vector<uint8_t> output_bytes_;
+};
+
+using TestDeltaByteArrayEncodingTypes = ::testing::Types<ByteArrayType, FLBAType>;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, /*repeats=*/0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 5));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 1));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_probability*/
+      0));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.5));
+}
+
+template <typename Type>
+class DeltaByteArrayEncodingDirectPut : public TestEncodingBase<Type> {
+ public:
+  std::unique_ptr<TypedEncoder<Type>> encoder =
+      MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+  std::unique_ptr<TypedDecoder<Type>> decoder =
+      MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+
+  void CheckDirectPut(std::shared_ptr<::arrow::Array> array) {
+    ASSERT_NO_THROW(encoder->Put(*array));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(array->length() - array->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<Type>::Accumulator acc;
+    using BuilderType = typename EncodingTraits<Type>::BuilderType;
+    acc.builder = std::make_unique<BuilderType>(array->type(), default_memory_pool());
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(array->length()),
+                                   static_cast<int>(array->null_count()),
+                                   array->null_bitmap_data(), array->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(array->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    ::arrow::AssertArraysEqual(*array, *result);
+  }
+
+  void CheckRoundtripFLBA() {
+    constexpr int64_t kSize = 50;
+    constexpr int kSeed = 42;
+    constexpr int kByteWidth = 4;
+    ::arrow::random::RandomArrayGenerator rag{kSeed};
+    std::shared_ptr<::arrow::Array> values =
+        rag.FixedSizeBinary(/*size=*/0, /*byte_width=*/kByteWidth);
+    CheckDirectPut(values);
+
+    for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+      values = rag.FixedSizeBinary(kSize + seed, kByteWidth);
+      CheckDirectPut(values);
+    }
+  }
+
+  void CheckRoundtripByteArray() {
+    constexpr int64_t kSize = 500;
+    constexpr int32_t kMinLength = 0;
+    constexpr int32_t kMaxLength = 10;
+    constexpr int32_t kNumUnique = 10;
+    constexpr double kNullProbability = 0.25;
+    constexpr int kSeed = 42;
+    ::arrow::random::RandomArrayGenerator rag{kSeed};
+    std::shared_ptr<::arrow::Array> values = rag.BinaryWithRepeats(
+        /*size=*/1, /*unique=*/1, kMinLength, kMaxLength, kNullProbability);
+    CheckDirectPut(values);
+
+    for (int i = 0; i < 10; ++i) {
+      values = rag.BinaryWithRepeats(kSize, kNumUnique, kMinLength, kMaxLength,
+                                     kNullProbability);
+      CheckDirectPut(values);
+    }
+  }
+
+  void CheckRoundtrip() override {
+    using ArrowType = typename EncodingTraits<Type>::ArrowType;
+    using IsFixedSizeBinary = ::arrow::is_fixed_size_binary_type<ArrowType>;
+
+    if constexpr (IsFixedSizeBinary::value) {
+      CheckRoundtripFLBA();
+    } else {
+      CheckRoundtripByteArray();
+    }
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+TYPED_TEST_SUITE(DeltaByteArrayEncodingDirectPut, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(DeltaByteArrayEncodingDirectPut, DirectPut) {
+  ASSERT_NO_FATAL_FAILURE(this->CheckRoundtrip());
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
+  auto CheckEncode = [](const std::shared_ptr<::arrow::Array>& values,
+                        const std::shared_ptr<Buffer>& encoded) {
+    auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+    ASSERT_TRUE(encoded->Equals(*buf));
+  };
+
+  auto ArrayToInt32Vector = [](const std::shared_ptr<::arrow::Array>& lengths) {
+    std::vector<int32_t> vector;
+    auto data_ptr = checked_cast<::arrow::Int32Array*>(lengths.get());
+    for (int i = 0; i < lengths->length(); ++i) {
+      vector.push_back(data_ptr->GetView(i));
+    }
+    return vector;
+  };
+
+  auto CheckDecode = [](std::shared_ptr<Buffer> buf,
+                        std::shared_ptr<::arrow::Array> values) {
+    int num_values = static_cast<int>(values->length());
+    auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(num_values, result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *upcast_result);
+  };
+
+  auto CheckEncodeDecode =
+      [&](std::string_view values, std::shared_ptr<::arrow::Array> prefix_lengths,
+          std::shared_ptr<::arrow::Array> suffix_lengths, std::string_view suffix_data) {
+        auto encoded =
+            ::arrow::ConcatenateBuffers({DeltaEncode(ArrayToInt32Vector(prefix_lengths)),

Review Comment:
   Is this approximately what you had in mind? https://github.com/apache/arrow/pull/14341/commits/d808c5a8df6fa520064872760e0ec224248efe68



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou merged pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou merged PR #14341:
URL: https://github.com/apache/arrow/pull/14341


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] wgtmac commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1623911055

   I will take a look in depth over the weekend. Hope it can be included in the 13.0.0 release as we have been wishing it for a long time.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1122037881


##########
cpp/src/parquet/encoding.cc:
##########
@@ -2900,6 +2900,124 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+ public:
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr, MemoryPool* pool)
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY,
+                    pool = ::arrow::default_memory_pool()),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override {
+    AssertBaseBinary(values);
+    const auto& data = values.data();
+    auto src = data->GetValues<T>(1);
+
+    if (values.null_count() == 0) {
+      Put(src, static_cast<int>(values.length()));
+    } else {
+      PutSpaced(src, static_cast<int>(data->length), data->GetValues<uint8_t>(0, 0),
+                data->offset);
+    }
+  }
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+  uint32_t total_value_count_{0};
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  string_view last_value_;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values);
+
+  if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+    last_value_ = string_view{reinterpret_cast<const char*>(src[0].ptr), src[0].len};
+    suffix_encoder_.Put(&src[0], 1);
+    prefix_lengths[0] = 0;
+  }
+  total_value_count_ += num_values;
+
+  for (int32_t i = 1; i < num_values; i++) {
+    auto value = string_view{reinterpret_cast<const char*>(src[i].ptr), src[i].len};
+
+    uint j = 0;
+    while (j < std::min(src[i - 1].len, src[i].len)) {
+      if (last_value_[j] != value[j]) {
+        break;
+      }
+      j++;
+    }
+
+    suffix_encoder_.Put(&src[i] + j, 1);

Review Comment:
   That makes sense, thanks! :D



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1145562983


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3265,6 +3420,14 @@ std::unique_ptr<Encoder> MakeEncoder(Type::type type_num, Encoding::type encodin
       default:
         throw ParquetException("DELTA_LENGTH_BYTE_ARRAY only supports BYTE_ARRAY");
     }
+  } else if (encoding == Encoding::DELTA_BYTE_ARRAY) {
+    switch (type_num) {
+      case Type::BYTE_ARRAY:
+        return std::make_unique<DeltaByteArrayEncoder<ByteArrayType>>(descr, pool);
+      default:
+        throw ParquetException("DELTA_BYTE_ARRAY only supports BYTE_ARRAY");

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1480564576

   Personally, I think we can make it easier, like using `PutImpl(std::vector<ByteArray>, int num_values)`. When Put with arrow, we can extract it into it.
   
   Or we can:
   
   ```C++
   template <typename ParquetT>
   struct ParquetIndirectType {
     static const uint8_t* getPtr(ParquetT object) noexcept;
     static int getLen(const parquet::ColumnDescriptor* descr, ParquetT object) noexcept;
   };
   
   template <>
   struct ParquetIndirectType<parquet::ByteArray> {
     static const uint8_t* getPtr(parquet::ByteArray object) noexcept {
       return object.ptr;
     }
   
     static int getLen([[maybe_unused]] const parquet::ColumnDescriptor* descr, parquet::ByteArray object) noexcept {
       return object.len;
     }
   };
   
   template <>
   struct ParquetIndirectType<parquet::FixedLenByteArray> {
     static const uint8_t* getPtr(parquet::FixedLenByteArray object) noexcept {
       return object.ptr;
     }
   
     static int getLen(
         const parquet::ColumnDescriptor* descr, [[maybe_unused]] parquet::FixedLenByteArray object) noexcept {
       return descr->type_length();
     }
   };
   ```
   
   And use template to just write one part of the code. But it may have worse performance than your impl if compiler doesn't work well. The current skeleton is ok to me


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1148416080


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3196,6 +3397,30 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) {
+    return GetInternal(buffer, max_values);
+  }
+  int Decode(FixedLenByteArray* buffer, int max_values) override {

Review Comment:
   `GetInternal` only works with `ByteArray` at the moment. As serialization goes via ByteArray anyway I'm not sure if it's worth supporting an additional path. Maybe just cast instead?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1285951190


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -874,7 +874,7 @@ std::shared_ptr<::arrow::Array> EncodingAdHocTyped<FLBAType>::GetValues(int seed
 }
 
 using EncodingAdHocTypedCases =
-    ::testing::Types<BooleanType, Int32Type, Int64Type, FloatType, DoubleType, FLBAType>;
+    ::testing::Types<BooleanType, Int32Type, Int64Type, FloatType, DoubleType>;

Review Comment:
   Yes, the change to `EncodingTraits<FLBAType>` breaks `EncodingAdHocTyped` test. This is to make it's interface equal to the `EncodingTraits<ByteArrayType>` one to enable templating. Specifically we go from:
   ```cpp
   struct EncodingTraits<FLBAType> {
     using Accumulator = ::arrow::FixedSizeBinaryBuilder;
     [...]
   ```
   to:
   ```cpp
   struct EncodingTraits<FLBAType> {
     struct Accumulator {
       std::unique_ptr<::arrow::FixedSizeBinaryBuilder> builder;
       std::vector<std::shared_ptr<::arrow::Array>> chunks;
     [...]
   ```
   If we consider `EncodingTraits` external API we should probably avoid this change.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1668302595

   Thanks for the suggestions @mapleFU ! As for your third point see https://github.com/apache/arrow/pull/14341#discussion_r1286004321.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1295904848


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1908,4 +1909,249 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestDeltaLengthByteArrayEncoding<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+  static constexpr double prefixed_probability = 0.5;
+
+  void InitData(int nvalues, int repeats) {
+    num_values_ = nvalues * repeats;
+    input_bytes_.resize(num_values_ * sizeof(c_type));
+    output_bytes_.resize(num_values_ * sizeof(c_type));
+    draws_ = reinterpret_cast<c_type*>(input_bytes_.data());
+    decode_buf_ = reinterpret_cast<c_type*>(output_bytes_.data());
+    GeneratePrefixedData<c_type>(nvalues, draws_, &data_buffer_, prefixed_probability);
+
+    // add some repeated values
+    for (int j = 1; j < repeats; ++j) {
+      for (int i = 0; i < nvalues; ++i) {
+        draws_[nvalues * j + i] = draws_[i];
+      }
+    }
+  }
+
+  Encoding::type GetEncoding() override { return Encoding::DELTA_BYTE_ARRAY; }
+
+ protected:
+  USING_BASE_MEMBERS();
+  std::vector<uint8_t> input_bytes_;
+  std::vector<uint8_t> output_bytes_;
+};
+
+using TestDeltaByteArrayEncodingTypes = ::testing::Types<ByteArrayType, FLBAType>;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, /*repeats=*/0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 5));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 1));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_probability*/
+      0));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.5));
+}
+
+template <typename Type>
+class DeltaByteArrayEncodingDirectPut : public TestEncodingBase<Type> {
+ public:
+  std::unique_ptr<TypedEncoder<Type>> encoder =
+      MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+  std::unique_ptr<TypedDecoder<Type>> decoder =
+      MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+
+  void CheckDirectPut(std::shared_ptr<::arrow::Array> array) {
+    ASSERT_NO_THROW(encoder->Put(*array));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(array->length() - array->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<Type>::Accumulator acc;
+    using BuilderType = typename EncodingTraits<Type>::BuilderType;
+    acc.builder = std::make_unique<BuilderType>(array->type(), default_memory_pool());
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(array->length()),
+                                   static_cast<int>(array->null_count()),
+                                   array->null_bitmap_data(), array->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(array->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    ::arrow::AssertArraysEqual(*array, *result);
+  }
+
+  void CheckRoundtripFLBA() {
+    constexpr int64_t kSize = 50;
+    constexpr int kSeed = 42;
+    constexpr int kByteWidth = 4;
+    ::arrow::random::RandomArrayGenerator rag{kSeed};
+    std::shared_ptr<::arrow::Array> values =
+        rag.FixedSizeBinary(/*size=*/0, /*byte_width=*/kByteWidth);
+    CheckDirectPut(values);
+
+    for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+      values = rag.FixedSizeBinary(kSize + seed, kByteWidth);
+      CheckDirectPut(values);
+    }
+  }
+
+  void CheckRoundtripByteArray() {
+    constexpr int64_t kSize = 500;
+    constexpr int32_t kMinLength = 0;
+    constexpr int32_t kMaxLength = 10;
+    constexpr int32_t kNumUnique = 10;
+    constexpr double kNullProbability = 0.25;
+    constexpr int kSeed = 42;
+    ::arrow::random::RandomArrayGenerator rag{kSeed};
+    std::shared_ptr<::arrow::Array> values = rag.BinaryWithRepeats(
+        /*size=*/1, /*unique=*/1, kMinLength, kMaxLength, kNullProbability);
+    CheckDirectPut(values);
+
+    for (int i = 0; i < 10; ++i) {
+      values = rag.BinaryWithRepeats(kSize, kNumUnique, kMinLength, kMaxLength,
+                                     kNullProbability);
+      CheckDirectPut(values);
+    }
+  }
+
+  void CheckRoundtrip() override {
+    using ArrowType = typename EncodingTraits<Type>::ArrowType;
+    using IsFixedSizeBinary = ::arrow::is_fixed_size_binary_type<ArrowType>;
+
+    if constexpr (IsFixedSizeBinary::value) {
+      CheckRoundtripFLBA();
+    } else {
+      CheckRoundtripByteArray();
+    }
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+TYPED_TEST_SUITE(DeltaByteArrayEncodingDirectPut, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(DeltaByteArrayEncodingDirectPut, DirectPut) {
+  ASSERT_NO_FATAL_FAILURE(this->CheckRoundtrip());
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
+  auto CheckEncode = [](const std::shared_ptr<::arrow::Array>& values,
+                        const std::shared_ptr<Buffer>& encoded) {
+    auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+    ASSERT_TRUE(encoded->Equals(*buf));
+  };
+
+  auto ArrayToInt32Vector = [](const std::shared_ptr<::arrow::Array>& lengths) {
+    std::vector<int32_t> vector;
+    auto data_ptr = checked_cast<::arrow::Int32Array*>(lengths.get());
+    for (int i = 0; i < lengths->length(); ++i) {
+      vector.push_back(data_ptr->GetView(i));
+    }
+    return vector;
+  };
+
+  auto CheckDecode = [](std::shared_ptr<Buffer> buf,
+                        std::shared_ptr<::arrow::Array> values) {
+    int num_values = static_cast<int>(values->length());
+    auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(num_values, result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *upcast_result);
+  };
+
+  auto CheckEncodeDecode =
+      [&](std::string_view values, std::shared_ptr<::arrow::Array> prefix_lengths,
+          std::shared_ptr<::arrow::Array> suffix_lengths, std::string_view suffix_data) {
+        auto encoded =
+            ::arrow::ConcatenateBuffers({DeltaEncode(ArrayToInt32Vector(prefix_lengths)),

Review Comment:
   Just an optional suggestion, but you could rewrite `DeltaEncode` to something like:
   ```c++
   std::shared_ptr<Buffer> DeltaEncode(::arrow::util::span<int32_t> lengths) {
     auto encoder = MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
     encoder->Put(lengths.data(), static_cast<int>(lengths.size()));
     return encoder->FlushValues();
   }
   
   std::shared_ptr<Buffer> DeltaEncode(const Int32Array& lengths) {
     return DeltaEncode(lengths.raw_values(), lengths.length());
   }
   ```
   
   and then you don't need `ArrayToInt32Vector` anymore.
   
   (btw, `arrow::util::span` is a backport of [`std::span`](https://en.cppreference.com/w/cpp/container/span))
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1295946043


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3063,241 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+  static constexpr std::string_view kEmpty = "";
+
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(/*descr=*/nullptr, pool),
+        suffix_encoder_(descr, pool),
+        last_value_(""),
+        empty_(static_cast<uint32_t>(kEmpty.size()),
+               reinterpret_cast<const uint8_t*>(kEmpty.data())) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != nullptr) {
+      if (buffer_ == nullptr) {
+        PARQUET_ASSIGN_OR_THROW(buffer_,
+                                ::arrow::AllocateResizableBuffer(num_values * sizeof(T),
+                                                                 this->memory_pool()));
+      } else {
+        PARQUET_THROW_NOT_OK(buffer_->Resize(num_values * sizeof(T), false));
+      }
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t flba_len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, flba_len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        const int idx = i + j;
+        const auto view = visitor[idx];
+        const uint32_t len = static_cast<uint32_t>(view.length());
+
+        uint32_t common_prefix_length = 0;
+        const uint32_t maximum_common_prefix_length =
+            std::min(len, static_cast<uint32_t>(last_value_view.length()));
+        while (common_prefix_length < maximum_common_prefix_length) {
+          if (last_value_view[common_prefix_length] != view[common_prefix_length]) {
+            break;
+          }
+          common_prefix_length++;
+        }
+
+        last_value_view = view;
+        prefix_lengths[j] = common_prefix_length;
+        const uint32_t suffix_length = len - common_prefix_length;
+        const uint8_t* suffix_ptr = src[idx].ptr + common_prefix_length;
+
+        // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+        const ByteArray suffix(suffix_length, suffix_ptr);
+        suffixes[j] = suffix;
+      }
+      suffix_encoder_.Put(suffixes.data(), batch_size);
+      prefix_length_encoder_.Put(prefix_lengths.data(), batch_size);
+    }
+    last_value_ = last_value_view;
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.length());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.

Review Comment:
   This comment can be removed as `src` is not passed to any encoder below.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1299984318


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,43 +1240,93 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
-  explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
-    this->out = out;
-    this->builder = out->builder.get();
-    this->chunk_space_remaining =
-        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+template <typename DType, typename Enable = void>
+struct ArrowBinaryHelper;
+
+template <typename DType>
+struct ArrowBinaryHelper<DType,
+                         std::enable_if_t<std::is_same_v<DType, ByteArrayType>, void>> {
+  explicit ArrowBinaryHelper(typename EncodingTraits<DType>::Accumulator* acc) {
+    builder = acc->builder.get();
+    chunks = &acc->chunks;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 builder->value_data_length(),
+                                                 &chunk_space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<DType>");
+    }
   }
 
   Status PushChunk() {
     std::shared_ptr<::arrow::Array> result;
     RETURN_NOT_OK(builder->Finish(&result));
-    out->chunks.push_back(result);
+    chunks->push_back(result);
     chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
     return Status::OK();
   }
 
   bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
 
   void UnsafeAppend(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
     builder->UnsafeAppend(data, length);
   }
 
   void UnsafeAppendNull() { builder->UnsafeAppendNull(); }
 
   Status Append(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
     return builder->Append(data, length);
   }
 
   Status AppendNull() { return builder->AppendNull(); }
 
-  typename EncodingTraits<ByteArrayType>::Accumulator* out;
-  ::arrow::BinaryBuilder* builder;
+  typename EncodingTraits<DType>::BuilderType* builder;
+  std::vector<std::shared_ptr<::arrow::Array>>* chunks;
   int64_t chunk_space_remaining;
 };
 
+template <typename DType>
+struct ArrowBinaryHelper<DType, std::enable_if_t<std::is_same_v<DType, FLBAType>, void>> {
+  explicit ArrowBinaryHelper(typename EncodingTraits<DType>::Accumulator* acc) {
+    builder = acc;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 builder->value_data_length(),
+                                                 &space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<DType>");
+    }
+  }
+
+  Status PushChunk() {
+    std::shared_ptr<::arrow::Array> result;
+    RETURN_NOT_OK(builder->Finish(&result));

Review Comment:
   See https://github.com/apache/arrow/pull/14341/commits/8f4a54adec0ef4ba7e2d3cc4cae1b5098a172034.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1686721655

   Thats a long time, bravo!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1686911907

   Thanks all for helping this along! I'm very happy we got this in!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1198509202


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,229 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        auto view = visitor[i + j];
+        len = visitor.len(i + j);
+
+        uint32_t k = 0;
+        const uint32_t common_length =
+            std::min(len, static_cast<uint32_t>(last_value_view.length()));
+        while (k < common_length) {
+          if (last_value_view[k] != view[k]) {
+            break;
+          }
+          k++;
+        }
+
+        last_value_view = view;
+        prefix_lengths[j] = k;
+        const auto suffix_length = len - k;
+        const uint8_t* suffix_ptr = src[i + j].ptr + k;
+
+        // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+        const ByteArray suffix(suffix_length, suffix_ptr);
+        suffixes[j] = suffix;
+      }
+      suffix_encoder_.Put(suffixes.data(), batch_size);
+      prefix_length_encoder_.Put(prefix_lengths.data(), batch_size);
+    }
+    last_value_ = last_value_view;
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+struct ByteArrayVisitor {
+  const ByteArray* src;
+  const uint32_t length;
+
+  std::string_view operator[](int i) const {
+    if (ARROW_PREDICT_FALSE(src[i].len >= kMaxByteArraySize)) {
+      throw ParquetException("Parquet cannot store strings with size 2GB or more");
+    }
+    return std::string_view{src[i]};
+  }
+
+  const uint32_t len(int i) const { return src[i].len; }
+};
+
+struct FLBAVisitor {
+  const FLBA* src;
+  const uint32_t length;
+
+  std::string_view operator[](int i) const {
+    return std::string_view{reinterpret_cast<const char*>(src[i].ptr)};

Review Comment:
   You should explicit set the length for flba, otherwise I think it will be regard as C style string and truncate the content after `\0`
   
   ```c++
     std::string_view operator[](int i) const {
       return std::string_view{reinterpret_cast<const char*>(src[i].ptr, length)};
     }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1198829839


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,229 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        auto view = visitor[i + j];
+        len = visitor.len(i + j);
+
+        uint32_t k = 0;
+        const uint32_t common_length =
+            std::min(len, static_cast<uint32_t>(last_value_view.length()));
+        while (k < common_length) {
+          if (last_value_view[k] != view[k]) {
+            break;
+          }
+          k++;
+        }
+
+        last_value_view = view;
+        prefix_lengths[j] = k;
+        const auto suffix_length = len - k;
+        const uint8_t* suffix_ptr = src[i + j].ptr + k;
+
+        // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+        const ByteArray suffix(suffix_length, suffix_ptr);
+        suffixes[j] = suffix;
+      }
+      suffix_encoder_.Put(suffixes.data(), batch_size);
+      prefix_length_encoder_.Put(prefix_lengths.data(), batch_size);
+    }
+    last_value_ = last_value_view;
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+struct ByteArrayVisitor {
+  const ByteArray* src;
+  const uint32_t length;
+
+  std::string_view operator[](int i) const {
+    if (ARROW_PREDICT_FALSE(src[i].len >= kMaxByteArraySize)) {
+      throw ParquetException("Parquet cannot store strings with size 2GB or more");
+    }
+    return std::string_view{src[i]};
+  }
+
+  uint32_t len(int i) const { return src[i].len; }
+};
+
+struct FLBAVisitor {
+  const FLBA* src;
+  const uint32_t length;

Review Comment:
   Seems this `length` means `type_length`, can you explicit mark that is type_length to avoid misleading? Seems that it looks like `length of FLBA array`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1571730242

   @rok Is this ready to review again?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1631762811

   @pitrou I think segfaults are now fixed (CI is failing on an abeil issue). The only thing I want to add is `DirectPut` test for `FLBAType`. The rest should be ready for review and post release merge :).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1633428478

   @pitrou I think this is now ready for review.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1269627474


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1908,4 +1907,304 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void InitData(int nvalues, int repeats, double prefixed_probability) {
+    num_values_ = nvalues * repeats;
+    input_bytes_.resize(num_values_ * sizeof(c_type));
+    output_bytes_.resize(num_values_ * sizeof(c_type));
+    draws_ = reinterpret_cast<c_type*>(input_bytes_.data());
+    decode_buf_ = reinterpret_cast<c_type*>(output_bytes_.data());
+    GeneratePrefixedData<c_type>(nvalues, draws_, &data_buffer_, prefixed_probability);
+
+    // add some repeated values
+    for (int j = 1; j < repeats; ++j) {
+      for (int i = 0; i < nvalues; ++i) {
+        draws_[nvalues * j + i] = draws_[i];
+      }
+    }
+  }
+
+  void Execute(int nvalues, int repeats, double prefixed_probability) {
+    InitData(nvalues, repeats, prefixed_probability);
+    CheckRoundtrip();
+  }
+
+  void ExecuteSpaced(int nvalues, int repeats, int64_t valid_bits_offset,
+                     double null_probability, double prefixed_probability) {
+    InitData(nvalues, repeats, prefixed_probability);
+
+    int64_t size = num_values_ + valid_bits_offset;
+    auto rand = ::arrow::random::RandomArrayGenerator(1923);
+    const auto array = rand.UInt8(size, /*min=*/0, /*max=*/100, null_probability);
+    const auto valid_bits = array->null_bitmap_data();
+    if (valid_bits) {
+      CheckRoundtripSpaced(valid_bits, valid_bits_offset);
+    }
+  }
+
+  void CheckRoundtrip() override {

Review Comment:
   I'll abstract these soon and ping back.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1182935098


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,25 +1240,35 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
+template <typename DType>
+struct ArrowBinaryHelper;
+
+template <>
+struct ArrowBinaryHelper<ByteArrayType> {
   explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
     this->out = out;
     this->builder = out->builder.get();
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 this->builder->value_data_length(),
+                                                 &this->chunk_space_remaining))) {
+      throw ParquetException("excess expansion in DELTA_LENGTH_BYTE_ARRAY");

Review Comment:
   Changed to:
   ```
   throw ParquetException("excess expansion in ArrowBinaryHelper<ByteArrayType>");
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1157594293


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,182 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {

Review Comment:
   It seems adding `DeltaByteArrayEncoder<FLBAType>::Put(const FLBA* src, int num_values)` solved this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1255050887


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,293 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void CheckRoundtrip() override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType, FLBAType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 2));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_prob*/ 0));
+
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 200));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.1));
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPut) {
+  const int64_t size = 50;
+  const int32_t min_length = 0;
+  const int32_t max_length = 10;
+  const int32_t num_unique = 10;
+  const double null_probability = 0.25;
+  auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  ::arrow::random::RandomArrayGenerator rag(42);
+  auto values = rag.String(0, min_length, max_length, null_probability);

Review Comment:
   I'd like to use this approach https://github.com/apache/arrow/pull/14341/files#diff-e050ca5e7d42fa0b87e5f0908c70f69d5bd795d12563045eb0927b668b6c476aR1921-R1956, however data generated this way is not read by the visitor correctly. I suppose it's not laid our correctly.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1256692220


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,25 +1240,35 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
+template <typename DType>
+struct ArrowBinaryHelper;
+
+template <>
+struct ArrowBinaryHelper<ByteArrayType> {
   explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
     this->out = out;
     this->builder = out->builder.get();
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 this->builder->value_data_length(),
+                                                 &this->chunk_space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<ByteArrayType>");
+    }
     this->chunk_space_remaining =

Review Comment:
   Good catch! Removing.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1627391847

   @mapleFU Please don't push for reviews. We are nearing a release and our priority is not to add major last-minute features.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1286005515


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),

Review Comment:
   Duplicate: https://github.com/apache/arrow/pull/14341/files#r1286004321



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: ARROW-17619: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by GitBox <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1271095563

   This is not really review ready yet and needs https://github.com/apache/arrow/pull/14191 and https://github.com/apache/arrow/pull/14293 to merge first.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1147066648


##########
cpp/src/parquet/encoding.cc:
##########
@@ -2900,8 +2929,222 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+ public:
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr, MemoryPool* pool)
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY,
+                    pool = ::arrow::default_memory_pool()),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+
+    // TODO(rok): optimize using ArrowPoolVector<int32_t> prefix_lengths(num_values);
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          uint32_t previous_len = 0;
+          const ByteArray src{view};

Review Comment:
   Great! Lets resolve this



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1122203542


##########
cpp/src/parquet/encoding.cc:
##########
@@ -2900,6 +2900,124 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+ public:
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr, MemoryPool* pool)
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY,
+                    pool = ::arrow::default_memory_pool()),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override {
+    AssertBaseBinary(values);
+    const auto& data = values.data();

Review Comment:
   Indeed. Added.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1144172826


##########
cpp/src/parquet/encoding.cc:
##########
@@ -2900,8 +2929,222 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+ public:
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr, MemoryPool* pool)
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY,
+                    pool = ::arrow::default_memory_pool()),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+
+    // TODO(rok): optimize using ArrowPoolVector<int32_t> prefix_lengths(num_values);
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          uint32_t previous_len = 0;
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+          } else {
+            uint32_t j = 0;
+            while (j < std::min(previous_len, byte_width)) {

Review Comment:
   Changed to `while (j < previous_len) {`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1142865848


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3265,6 +3420,14 @@ std::unique_ptr<Encoder> MakeEncoder(Type::type type_num, Encoding::type encodin
       default:
         throw ParquetException("DELTA_LENGTH_BYTE_ARRAY only supports BYTE_ARRAY");
     }
+  } else if (encoding == Encoding::DELTA_BYTE_ARRAY) {
+    switch (type_num) {
+      case Type::BYTE_ARRAY:
+        return std::make_unique<DeltaByteArrayEncoder<ByteArrayType>>(descr, pool);
+      default:
+        throw ParquetException("DELTA_BYTE_ARRAY only supports BYTE_ARRAY");

Review Comment:
   Added! Templateing could perhaps be done in a nicer way.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1153907155


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          const uint8_t* suffix_ptr = src.ptr + j;
+          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+          last_value_view = view;
+          // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  int i = 0;
+  while (i < num_values) {
+    // Convert to ByteArray so we can pass to the suffix_encoder_.
+    auto value = reinterpret_cast<const ByteArray*>(&src[i]);
+    if (ARROW_PREDICT_FALSE(value->len >= kMaxByteArraySize)) {
+      throw Status::Invalid("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value->ptr), value->len};
+    uint32_t j = 0;
+    while (j < std::min(value->len, static_cast<uint32_t>(last_value_view.length()))) {

Review Comment:
   It's definetly better for readability!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1491492079

   Thanks for @mapleFU. I used your suggestion.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1197179358


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,12 +3079,243 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))),
+        prefix_lengths_(kBatchSize_, ::arrow::stl::allocator<int32_t>(pool_)) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+  ArrowPoolVector<int32_t> prefix_lengths_;

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1154932762


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          const uint8_t* suffix_ptr = src.ptr + j;
+          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+          last_value_view = view;
+          // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  int i = 0;
+  while (i < num_values) {
+    // Convert to ByteArray so we can pass to the suffix_encoder_.
+    auto value = reinterpret_cast<const ByteArray*>(&src[i]);
+    if (ARROW_PREDICT_FALSE(value->len >= kMaxByteArraySize)) {
+      throw Status::Invalid("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value->ptr), value->len};
+    uint32_t j = 0;
+    while (j < std::min(value->len, static_cast<uint32_t>(last_value_view.length()))) {

Review Comment:
   Ah, but it's still not fixed. I'll take a look this tomorrow.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1154931866


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          const uint8_t* suffix_ptr = src.ptr + j;
+          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+          last_value_view = view;
+          // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  int i = 0;
+  while (i < num_values) {
+    // Convert to ByteArray so we can pass to the suffix_encoder_.
+    auto value = reinterpret_cast<const ByteArray*>(&src[i]);
+    if (ARROW_PREDICT_FALSE(value->len >= kMaxByteArraySize)) {
+      throw Status::Invalid("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value->ptr), value->len};
+    uint32_t j = 0;
+    while (j < std::min(value->len, static_cast<uint32_t>(last_value_view.length()))) {

Review Comment:
   Yeah, that seems to be the case.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1571759460

   @pitrou I believe I need to finish the random array generator. I got sidetracked on a construction project and I'll try to finish this by end of the week.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1256731370


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,230 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1256731195


##########
cpp/src/parquet/encoding.cc:
##########
@@ -2842,7 +2888,7 @@ class DeltaLengthByteArrayDecoder : public DecoderImpl,
 
   std::shared_ptr<::arrow::bit_util::BitReader> decoder_;
   DeltaBitPackDecoder<Int32Type> len_decoder_;
-  int num_valid_values_;
+  int num_valid_values_{0};

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1630873381

   There are CI failures. At least the Conda C++ and ASAN/UBSAN ones should be easily reproducer using Docker.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1257304151


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1910,5 +1909,310 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void InitData(int nvalues, double null_probability) {
+    const int seed = 42;
+    auto rand = ::arrow::random::RandomArrayGenerator(seed);
+    const int min_prefix_length = 0;
+    const int max_prefix_length = 100;
+    const int max_element_length = 1000;
+    const double prefixed_probability = 0.5;
+
+    const auto prefix_array = std::dynamic_pointer_cast<::arrow::StringArray>(
+        rand.String(nvalues, min_prefix_length, max_prefix_length, null_probability));
+    const auto do_prefix = std::dynamic_pointer_cast<::arrow::BooleanArray>(
+        rand.Boolean(nvalues,
+                     /*true_probability=*/prefixed_probability,
+                     /*null_probability=*/0.0));
+    ::arrow::StringBuilder builder(::arrow::default_memory_pool());
+
+    std::string prefix = "";
+    for (int i = 0; i < nvalues; i++) {
+      if (prefix_array->IsNull(i)) {
+        PARQUET_THROW_NOT_OK(builder.AppendNull());
+      } else {
+        const std::string element = prefix_array->GetString(i);
+        if (do_prefix->Value(i) && prefix.length() < max_element_length) {
+          prefix = prefix.append(element);
+        } else {
+          prefix = element;
+        }
+        PARQUET_THROW_NOT_OK(builder.Append(prefix));
+      }
+    }
+
+    std::shared_ptr<::arrow::StringArray> array;
+    ASSERT_OK(builder.Finish(&array));
+    num_values_ = static_cast<int>(array->length() - array->null_count());
+    draws_ = reinterpret_cast<c_type*>(array->value_data()->mutable_data());
+  }
+
+  void Execute(int nvalues, double null_probability) {
+    InitData(nvalues, null_probability);
+    CheckRoundtrip();
+  }
+
+  void ExecuteSpaced(int nvalues, int repeats, int64_t valid_bits_offset,
+                     double null_probability) {
+    InitData(nvalues, null_probability);
+
+    int64_t size = num_values_ + valid_bits_offset;
+    auto rand = ::arrow::random::RandomArrayGenerator(1923);
+    const auto array = rand.UInt8(size, 0, 100, null_probability);
+    const auto valid_bits = array->null_bitmap_data();
+    if (valid_bits) {
+      CheckRoundtripSpaced(valid_bits, valid_bits_offset);
+    }
+  }
+
+  void CheckRoundtrip() override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+using TestDeltaByteArrayEncodingTypes = ::testing::Types<ByteArrayType, FLBAType>;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  // TODO
+
+  //  ASSERT_NO_FATAL_FAILURE(this->Execute(250, /*null_probability*/ 0));
+  //  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+  //      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_probability*/
+  //      0));
+  //
+  //  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, /*null_probability*/ 0.1));
+  //  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+  //      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+  //      /*null_probability*/ 0.5));

Review Comment:
   It would segfault at encoding time in `ByteArrayVisitor` at the moment (I assume due to the way data is generated). I'm looking into it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1269601956


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3058,237 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+  static constexpr std::string_view kEmpty = "";
+
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        // Prefix lengths are encoded using DeltaBitPackEncoder that can be left
+        // uninitialized.
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(descr, pool),
+        last_value_(""),
+        empty_(static_cast<uint32_t>(kEmpty.size()),

Review Comment:
   Unfortunately we can't make constructor `constexpr` because `reinterpret_cast` here:
   ```
     ByteArray(::std::string_view view)  // NOLINT implicit conversion
         : ByteArray(static_cast<uint32_t>(view.size()),
                     reinterpret_cast<const uint8_t*>(view.data())) {}
   ```
   is not allowed by `constexpr` as [per this](https://stackoverflow.com/questions/26200827/getting-around-the-reinterpret-cast-limitation-with-constexpr).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1270784709


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1908,4 +1907,304 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void InitData(int nvalues, int repeats, double prefixed_probability) {
+    num_values_ = nvalues * repeats;
+    input_bytes_.resize(num_values_ * sizeof(c_type));
+    output_bytes_.resize(num_values_ * sizeof(c_type));
+    draws_ = reinterpret_cast<c_type*>(input_bytes_.data());
+    decode_buf_ = reinterpret_cast<c_type*>(output_bytes_.data());
+    GeneratePrefixedData<c_type>(nvalues, draws_, &data_buffer_, prefixed_probability);
+
+    // add some repeated values
+    for (int j = 1; j < repeats; ++j) {
+      for (int i = 0; i < nvalues; ++i) {
+        draws_[nvalues * j + i] = draws_[i];
+      }
+    }
+  }
+
+  void Execute(int nvalues, int repeats, double prefixed_probability) {
+    InitData(nvalues, repeats, prefixed_probability);
+    CheckRoundtrip();
+  }
+
+  void ExecuteSpaced(int nvalues, int repeats, int64_t valid_bits_offset,
+                     double null_probability, double prefixed_probability) {
+    InitData(nvalues, repeats, prefixed_probability);
+
+    int64_t size = num_values_ + valid_bits_offset;
+    auto rand = ::arrow::random::RandomArrayGenerator(1923);
+    const auto array = rand.UInt8(size, /*min=*/0, /*max=*/100, null_probability);
+    const auto valid_bits = array->null_bitmap_data();
+    if (valid_bits) {
+      CheckRoundtripSpaced(valid_bits, valid_bits_offset);
+    }
+  }
+
+  void CheckRoundtrip() override {

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1285715935


##########
cpp/src/parquet/test_util.h:
##########
@@ -783,8 +789,33 @@ inline void GenerateData<ByteArray>(int num_values, ByteArray* out,
   random_byte_array(num_values, 0, heap->data(), out, 2, max_byte_array_len);
 }
 
+template <typename T>
+inline void GeneratePrefixedData(int num_values, T* out, std::vector<uint8_t>* heap,
+                                 double prefixed_probability);
+
+template <>
+inline void GeneratePrefixedData(int num_values, ByteArray* out,
+                                 std::vector<uint8_t>* heap,
+                                 double prefixed_probability) {
+  // seed the prng so failure is deterministic
+  int max_byte_array_len = 12;
+  heap->resize(num_values * max_byte_array_len);
+  prefixed_random_byte_array(num_values, 0, heap->data(), out, 2, max_byte_array_len,

Review Comment:
   ```suggestion
     prefixed_random_byte_array(num_values, /*seed=*/0, heap->data(), out, /*min_size=*/2, /*max_size=*/max_byte_array_len,
   ```



##########
cpp/src/parquet/test_util.h:
##########
@@ -783,8 +789,33 @@ inline void GenerateData<ByteArray>(int num_values, ByteArray* out,
   random_byte_array(num_values, 0, heap->data(), out, 2, max_byte_array_len);
 }
 
+template <typename T>
+inline void GeneratePrefixedData(int num_values, T* out, std::vector<uint8_t>* heap,
+                                 double prefixed_probability);
+
+template <>
+inline void GeneratePrefixedData(int num_values, ByteArray* out,
+                                 std::vector<uint8_t>* heap,
+                                 double prefixed_probability) {
+  // seed the prng so failure is deterministic
+  int max_byte_array_len = 12;
+  heap->resize(num_values * max_byte_array_len);
+  prefixed_random_byte_array(num_values, 0, heap->data(), out, 2, max_byte_array_len,
+                             prefixed_probability);
+}
+
 static constexpr int kGenerateDataFLBALength = 8;
 
+template <>
+inline void GeneratePrefixedData<FLBA>(int num_values, FLBA* out,
+                                       std::vector<uint8_t>* heap,
+                                       double prefixed_probability) {
+  // seed the prng so failure is deterministic
+  heap->resize(num_values * kGenerateDataFLBALength);
+  prefixed_random_byte_array(num_values, 0, heap->data(), kGenerateDataFLBALength, out,

Review Comment:
   ```suggestion
     prefixed_random_byte_array(num_values, /*seed=*/0, heap->data(), kGenerateDataFLBALength, out,
   ```



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3083,242 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          buffer_, ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));

Review Comment:
   1. `buffer_` will create everytime when `PutSpace` is called
   2. When `Flush` called , would `buffer_` been cleared?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1196626623


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,12 +3079,243 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))),
+        prefix_lengths_(kBatchSize_, ::arrow::stl::allocator<int32_t>(pool_)) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+  ArrowPoolVector<int32_t> prefix_lengths_;
+  static constexpr int kBatchSize_ = 256;
+};
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i += kBatchSize_) {
+    const int batch_size = std::min(kBatchSize_, num_values - i);
+
+    for (int j = 0; j < batch_size; ++j) {
+      // Convert to ByteArray, so we can pass to the suffix_encoder_.
+      const ByteArray value = src[i + j];
+      if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+        throw ParquetException("Parquet cannot store strings with size 2GB or more");
+      }
+      auto view = std::string_view{value};
+
+      uint32_t k = 0;
+      const uint32_t common_length =
+          std::min(value.len, static_cast<uint32_t>(last_value_view.length()));
+      while (k < common_length) {
+        if (last_value_view[k] != view[k]) {
+          break;
+        }
+        k++;
+      }
+
+      last_value_view = view;
+      prefix_lengths_[j] = k;
+      const auto suffix_length = static_cast<uint32_t>(value.len - k);
+
+      if (suffix_length == 0) {
+        continue;
+      }

Review Comment:
   I don't understand. You still need to write a 0-size string to the suffix encoder.
   ```suggestion
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1198509202


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,229 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        auto view = visitor[i + j];
+        len = visitor.len(i + j);
+
+        uint32_t k = 0;
+        const uint32_t common_length =
+            std::min(len, static_cast<uint32_t>(last_value_view.length()));
+        while (k < common_length) {
+          if (last_value_view[k] != view[k]) {
+            break;
+          }
+          k++;
+        }
+
+        last_value_view = view;
+        prefix_lengths[j] = k;
+        const auto suffix_length = len - k;
+        const uint8_t* suffix_ptr = src[i + j].ptr + k;
+
+        // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+        const ByteArray suffix(suffix_length, suffix_ptr);
+        suffixes[j] = suffix;
+      }
+      suffix_encoder_.Put(suffixes.data(), batch_size);
+      prefix_length_encoder_.Put(prefix_lengths.data(), batch_size);
+    }
+    last_value_ = last_value_view;
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+struct ByteArrayVisitor {
+  const ByteArray* src;
+  const uint32_t length;
+
+  std::string_view operator[](int i) const {
+    if (ARROW_PREDICT_FALSE(src[i].len >= kMaxByteArraySize)) {
+      throw ParquetException("Parquet cannot store strings with size 2GB or more");
+    }
+    return std::string_view{src[i]};
+  }
+
+  const uint32_t len(int i) const { return src[i].len; }
+};
+
+struct FLBAVisitor {
+  const FLBA* src;
+  const uint32_t length;
+
+  std::string_view operator[](int i) const {
+    return std::string_view{reinterpret_cast<const char*>(src[i].ptr)};

Review Comment:
   ```
     std::string_view operator[](int i) const {
       return std::string_view{reinterpret_cast<const char*>(src[i].ptr, length)};
     }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1688512151

   I created an issue :) https://github.com/apache/arrow/issues/37312


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1160575774


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3083,240 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          buffer_, ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  std::unique_ptr<::arrow::Buffer> buffer_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw Status::Invalid("Put not implemented for " + this->descr_->ToString());
+}
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i++) {
+    // Convert to ByteArray, so we can pass to the suffix_encoder_.
+    const ByteArray value = src[i];
+    if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+      throw Status::Invalid("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value.ptr),
+                            static_cast<uint32_t>(value.len)};
+    uint32_t j = 0;
+    const uint32_t common_length =
+        std::min(value.len, static_cast<uint32_t>(last_value_view.length()));
+    while (j < common_length) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
+    last_value_view = view;
+    prefix_lengths[i] = j;
+    const auto suffix_length = static_cast<uint32_t>(value.len - j);
+
+    if (suffix_length == 0) {
+      continue;

Review Comment:
   Done. I hope this doesn't trigger UBSAN again :D.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3083,240 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          buffer_, ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  std::unique_ptr<::arrow::Buffer> buffer_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw Status::Invalid("Put not implemented for " + this->descr_->ToString());
+}
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i++) {
+    // Convert to ByteArray, so we can pass to the suffix_encoder_.
+    const ByteArray value = src[i];
+    if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+      throw Status::Invalid("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value.ptr),
+                            static_cast<uint32_t>(value.len)};
+    uint32_t j = 0;
+    const uint32_t common_length =
+        std::min(value.len, static_cast<uint32_t>(last_value_view.length()));
+    while (j < common_length) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
+    last_value_view = view;
+    prefix_lengths[i] = j;
+    const auto suffix_length = static_cast<uint32_t>(value.len - j);
+
+    if (suffix_length == 0) {
+      continue;
+    }
+    const uint8_t* suffix_ptr = value.ptr + j;
+    // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+    const ByteArray suffix(suffix_length, suffix_ptr);
+    suffix_encoder_.Put(&suffix, 1);
+  }
+  prefix_length_encoder_.Put(prefix_lengths.data(), num_values);
+  last_value_ = last_value_view;
+}
+
+template <>
+void DeltaByteArrayEncoder<FLBAType>::Put(const FLBA* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+  const int32_t len = descr_->type_length();
+
+  if (ARROW_PREDICT_FALSE(len >= static_cast<int32_t>(kMaxByteArraySize))) {
+    throw Status::Invalid("Parquet cannot store strings with size 2GB or more");
+  }
+
+  for (int i = 0; i < num_values; i++) {
+    auto view = string_view{reinterpret_cast<const char*>(src[i].ptr),
+                            static_cast<uint32_t>(len)};
+    int32_t j = 0;
+    const int32_t common_length =
+        std::min(len, static_cast<int32_t>(last_value_view.length()));
+    while (j < common_length) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
+    last_value_view = view;
+    prefix_lengths[i] = j;
+    const auto suffix_length = static_cast<uint32_t>(len - j);
+
+    if (suffix_length == 0) {
+      continue;

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1158124125


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3149,9 +3160,14 @@ class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DT
           previous_len = src.len;
           prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
 
-          const uint8_t* suffix_ptr = src.ptr + j;
-          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
           last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          const uint8_t* suffix_ptr;
+          if (suffix_length == 0) {
+            suffix_ptr = reinterpret_cast<const uint8_t*>("");

Review Comment:
   Indeed! Switched to `nullptr`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] wgtmac commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1155415558


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1275,6 +1280,37 @@ struct ArrowBinaryHelper {
   int64_t chunk_space_remaining;
 };
 
+template <>
+struct ArrowBinaryHelper<FLBAType> {
+  explicit ArrowBinaryHelper(EncodingTraits<FLBAType>::Accumulator* builder) {
+    this->builder = builder;
+    this->chunk_space_remaining =

Review Comment:
   Is overflow check a good idea here and line 1250 above?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),

Review Comment:
   nit: `buffer` can be reused for better performance.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");

Review Comment:
   Do we have any evidence where does the `2GB` come from? Would be better to add a comment for the reason here.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          const uint8_t* suffix_ptr = src.ptr + j;
+          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+          last_value_view = view;
+          // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);

Review Comment:
   Is it an UB if `suffix_length == 0`? Should we pass a static empty string for safety?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          const uint8_t* suffix_ptr = src.ptr + j;
+          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+          last_value_view = view;
+          // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  int i = 0;
+  while (i < num_values) {
+    // Convert to ByteArray so we can pass to the suffix_encoder_.
+    auto value = reinterpret_cast<const ByteArray*>(&src[i]);

Review Comment:
   In this case, the length should be derived from the type.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,182 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {

Review Comment:
   I guess it may be caused by the cast at line 3183. FLBA type is not covered by any test yet. @wjones127 



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          const uint8_t* suffix_ptr = src.ptr + j;
+          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+          last_value_view = view;
+          // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  int i = 0;
+  while (i < num_values) {
+    // Convert to ByteArray so we can pass to the suffix_encoder_.
+    auto value = reinterpret_cast<const ByteArray*>(&src[i]);
+    if (ARROW_PREDICT_FALSE(value->len >= kMaxByteArraySize)) {
+      throw Status::Invalid("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value->ptr), value->len};
+    uint32_t j = 0;
+    while (j < std::min(value->len, static_cast<uint32_t>(last_value_view.length()))) {

Review Comment:
   last_value_view is not mutated in the current version, should we do what suggested above for better readability and performance?



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,285 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  virtual void CheckRoundtrip() {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits, int64_t valid_bits_offset) {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+// TODO: add FLBAType and Decimal type tests
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 2));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_prob*/ 0));
+
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 200));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.1));
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPut) {
+  const int64_t size = 50;
+  const int32_t min_length = 0;
+  const int32_t max_length = 10;
+  const int32_t num_unique = 10;
+  const double null_probability = 0.25;
+  auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  ::arrow::random::RandomArrayGenerator rag(42);
+  auto values = rag.String(0, min_length, max_length, null_probability);
+  CheckSeed(values);
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    rag = ::arrow::random::RandomArrayGenerator(seed);
+
+    values = rag.String(size, min_length, max_length, null_probability);
+    CheckSeed(values);
+
+    values =
+        rag.BinaryWithRepeats(size, num_unique, min_length, max_length, null_probability);
+    CheckSeed(values);
+  }
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPutFixedLength) {
+  const int64_t size = 50;
+  const double null_probability = 0.25;
+  ::arrow::random::RandomArrayGenerator rag(0);
+  auto encoder = MakeTypedEncoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<FLBAType>::Accumulator acc(values->type());
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    for (auto length : {0, 10, 100, 1000}) {
+      rag = ::arrow::random::RandomArrayGenerator(seed);
+      auto values = rag.FixedSizeBinary(size, length, null_probability);
+      CheckSeed(values);
+    }
+  }
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
+  auto CheckEncode = [](std::shared_ptr<::arrow::Array> values,
+                        std::shared_ptr<::arrow::Array> prefix_lengths,
+                        std::shared_ptr<::arrow::Array> suffix_lengths,
+                        std::string_view value) {
+    auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    auto prefix_lengths_encoder =
+        MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
+    ASSERT_NO_THROW(prefix_lengths_encoder->Put(*prefix_lengths));
+    auto prefix_lengths_buf = prefix_lengths_encoder->FlushValues();
+
+    auto encoded_prefix_lengths_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
+
+    auto suffix_lengths_encoder =
+        MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
+    ASSERT_NO_THROW(suffix_lengths_encoder->Put(*suffix_lengths));
+    auto suffix_lengths_buf = suffix_lengths_encoder->FlushValues();
+    auto encoded_values_buf =
+        SliceBuffer(buf, prefix_lengths_buf->size() + suffix_lengths_buf->size());
+
+    auto encoded_prefix_length_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
+    EXPECT_TRUE(prefix_lengths_buf->Equals(*encoded_prefix_length_buf));
+    auto encoded_suffix_length_buf =
+        SliceBuffer(buf, prefix_lengths_buf->size(), suffix_lengths_buf->size());
+    EXPECT_TRUE(suffix_lengths_buf->Equals(*encoded_suffix_length_buf));
+    EXPECT_EQ(value, encoded_values_buf->ToString());
+  };
+
+  auto arrayToI32 = [](const std::shared_ptr<::arrow::Array>& lengths) {
+    std::vector<int32_t> arrays;
+    auto data_ptr = checked_cast<::arrow::Int32Array*>(lengths.get());
+    for (int i = 0; i < lengths->length(); ++i) {
+      arrays.push_back(data_ptr->GetView(i));
+    }
+    return arrays;
+  };
+
+  auto CheckDecode = [](std::shared_ptr<Buffer> buf,
+                        std::shared_ptr<::arrow::Array> values) {
+    int num_values = static_cast<int>(values->length());
+    auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(num_values, result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *upcast_result);
+  };
+
+  auto checkEncodeDecode = [&](std::string_view values,
+                               std::shared_ptr<::arrow::Array> prefix_lengths,
+                               std::shared_ptr<::arrow::Array> suffix_lengths,
+                               std::string_view suffix_data) {
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::utf8(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::large_utf8(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::binary(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::large_binary(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+
+    auto encoded = ::arrow::ConcatenateBuffers({DeltaEncode(arrayToI32(prefix_lengths)),
+                                                DeltaEncode(arrayToI32(suffix_lengths)),
+                                                std::make_shared<Buffer>(suffix_data)})
+                       .ValueOrDie();
+
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::utf8(), values));
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_utf8(), values));
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::binary(), values));
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
+  };
+
+  {
+    auto values = R"(["axis", "axle", "babble", "babyhood"])";

Review Comment:
   Should we add a test for non-ASCII but UTF8-conforming characters?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          const uint8_t* suffix_ptr = src.ptr + j;
+          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+          last_value_view = view;
+          // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  int i = 0;
+  while (i < num_values) {
+    // Convert to ByteArray so we can pass to the suffix_encoder_.
+    auto value = reinterpret_cast<const ByteArray*>(&src[i]);

Review Comment:
   T is `FLBA` when DType is FLBAType, we cannot blindly cast it to `ByteArray`.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          const uint8_t* suffix_ptr = src.ptr + j;
+          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+          last_value_view = view;
+          // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  int i = 0;
+  while (i < num_values) {

Review Comment:
   We can use for loop instead of while loop since `i` is not used outside.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {

Review Comment:
   `std::min(previous_len, src.len)` can be put out of the while loop as it is constant.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1157447850


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),

Review Comment:
   Indeed. Added.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1160518706


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,25 +1240,35 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
+template <typename DType>
+struct ArrowBinaryHelper;
+
+template <>
+struct ArrowBinaryHelper<ByteArrayType> {
   explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
     this->out = out;
     this->builder = out->builder.get();
+    if (SubtractWithOverflow(::arrow::kBinaryMemoryLimit,

Review Comment:
   Mind add a `ARROW_PREDICT_FALSE`?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3083,240 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          buffer_, ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  std::unique_ptr<::arrow::Buffer> buffer_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw Status::Invalid("Put not implemented for " + this->descr_->ToString());
+}
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i++) {
+    // Convert to ByteArray, so we can pass to the suffix_encoder_.
+    const ByteArray value = src[i];
+    if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+      throw Status::Invalid("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value.ptr),
+                            static_cast<uint32_t>(value.len)};
+    uint32_t j = 0;
+    const uint32_t common_length =
+        std::min(value.len, static_cast<uint32_t>(last_value_view.length()));
+    while (j < common_length) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
+    last_value_view = view;
+    prefix_lengths[i] = j;
+    const auto suffix_length = static_cast<uint32_t>(value.len - j);
+
+    if (suffix_length == 0) {
+      continue;
+    }
+    const uint8_t* suffix_ptr = value.ptr + j;
+    // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+    const ByteArray suffix(suffix_length, suffix_ptr);
+    suffix_encoder_.Put(&suffix, 1);
+  }
+  prefix_length_encoder_.Put(prefix_lengths.data(), num_values);
+  last_value_ = last_value_view;
+}
+
+template <>
+void DeltaByteArrayEncoder<FLBAType>::Put(const FLBA* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+  const int32_t len = descr_->type_length();
+
+  if (ARROW_PREDICT_FALSE(len >= static_cast<int32_t>(kMaxByteArraySize))) {
+    throw Status::Invalid("Parquet cannot store strings with size 2GB or more");
+  }
+
+  for (int i = 0; i < num_values; i++) {
+    auto view = string_view{reinterpret_cast<const char*>(src[i].ptr),
+                            static_cast<uint32_t>(len)};
+    int32_t j = 0;
+    const int32_t common_length =
+        std::min(len, static_cast<int32_t>(last_value_view.length()));
+    while (j < common_length) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
+    last_value_view = view;
+    prefix_lengths[i] = j;
+    const auto suffix_length = static_cast<uint32_t>(len - j);
+
+    if (suffix_length == 0) {
+      continue;

Review Comment:
   ditto



##########
python/pyarrow/tests/parquet/test_basic.py:
##########
@@ -426,6 +430,21 @@ def test_column_encoding(use_legacy_dataset):
                                       'c': "DELTA_LENGTH_BYTE_ARRAY"},
                      use_legacy_dataset=use_legacy_dataset)
 
+    # Check "DELTA_BYTE_ARRAY" for byte columns.
+    _check_roundtrip(mixed_table, expected=mixed_table,
+                     use_dictionary=False,
+                     column_encoding={'a': "PLAIN",
+                                      'b': "DELTA_BINARY_PACKED",
+                                      'c': "DELTA_BYTE_ARRAY",
+                                      'd': "DELTA_BYTE_ARRAY"},
+                     use_legacy_dataset=use_legacy_dataset)
+
+    # Check "RLE" for boolean columns.

Review Comment:
   Thanks!



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3083,240 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          buffer_, ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  std::unique_ptr<::arrow::Buffer> buffer_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw Status::Invalid("Put not implemented for " + this->descr_->ToString());
+}
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i++) {
+    // Convert to ByteArray, so we can pass to the suffix_encoder_.
+    const ByteArray value = src[i];
+    if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+      throw Status::Invalid("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value.ptr),
+                            static_cast<uint32_t>(value.len)};
+    uint32_t j = 0;
+    const uint32_t common_length =
+        std::min(value.len, static_cast<uint32_t>(last_value_view.length()));
+    while (j < common_length) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
+    last_value_view = view;
+    prefix_lengths[i] = j;
+    const auto suffix_length = static_cast<uint32_t>(value.len - j);
+
+    if (suffix_length == 0) {
+      continue;

Review Comment:
   Here should put a `kEmpty`



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3196,6 +3467,43 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) {
+    return GetInternal(buffer, max_values);
+  }
+  int Decode(FixedLenByteArray* buffer, int max_values) override {
+    int decoded_values_size = max_values;
+    if (MultiplyWithOverflow(decoded_values_size,

Review Comment:
   Add a predict false here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer [arrow]

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1386842190


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1335,13 +1420,7 @@ class PlainByteArrayDecoder : public PlainDecoder<ByteArrayType>,
           if (ARROW_PREDICT_FALSE(len_ < increment)) {
             ParquetException::EofException();
           }
-          if (ARROW_PREDICT_FALSE(!helper.CanFit(value_len))) {
-            // This element would exceed the capacity of a chunk
-            RETURN_NOT_OK(helper.PushChunk());
-            RETURN_NOT_OK(helper.builder->Reserve(num_values - i));
-            RETURN_NOT_OK(helper.builder->ReserveData(
-                std::min<int64_t>(len_, helper.chunk_space_remaining)));

Review Comment:
   I also don't understand them T_T.
   
   Just confused by this change, so I tried to understand the origin code and find out why this cause the regression, what should I do to fix it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1299985895


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,43 +1240,93 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
-  explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
-    this->out = out;
-    this->builder = out->builder.get();
-    this->chunk_space_remaining =
-        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+template <typename DType, typename Enable = void>
+struct ArrowBinaryHelper;
+
+template <typename DType>
+struct ArrowBinaryHelper<DType,
+                         std::enable_if_t<std::is_same_v<DType, ByteArrayType>, void>> {
+  explicit ArrowBinaryHelper(typename EncodingTraits<DType>::Accumulator* acc) {
+    builder = acc->builder.get();
+    chunks = &acc->chunks;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 builder->value_data_length(),
+                                                 &chunk_space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<DType>");
+    }
   }
 
   Status PushChunk() {
     std::shared_ptr<::arrow::Array> result;
     RETURN_NOT_OK(builder->Finish(&result));
-    out->chunks.push_back(result);
+    chunks->push_back(result);
     chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
     return Status::OK();
   }
 
   bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
 
   void UnsafeAppend(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
     builder->UnsafeAppend(data, length);
   }
 
   void UnsafeAppendNull() { builder->UnsafeAppendNull(); }
 
   Status Append(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
     return builder->Append(data, length);
   }
 
   Status AppendNull() { return builder->AppendNull(); }
 
-  typename EncodingTraits<ByteArrayType>::Accumulator* out;
-  ::arrow::BinaryBuilder* builder;
+  typename EncodingTraits<DType>::BuilderType* builder;
+  std::vector<std::shared_ptr<::arrow::Array>>* chunks;
   int64_t chunk_space_remaining;
 };
 
+template <typename DType>
+struct ArrowBinaryHelper<DType, std::enable_if_t<std::is_same_v<DType, FLBAType>, void>> {
+  explicit ArrowBinaryHelper(typename EncodingTraits<DType>::Accumulator* acc) {
+    builder = acc;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 builder->value_data_length(),
+                                                 &space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<DType>");
+    }
+  }
+
+  Status PushChunk() {
+    std::shared_ptr<::arrow::Array> result;
+    RETURN_NOT_OK(builder->Finish(&result));

Review Comment:
   Another option would be to refactor `ArrowBinaryHelper` to something like `ArrowBinaryHelper<DType>(EncodingTraits<DType>::BuilderType, `std::vector<std::shared_ptr<::arrow::Array>>*`).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1483883942

   @mapleFU I've refactored to templates. Could you please do another pass?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1482208402

   Seems the skeleton look great now. I think we can extract `Put` because now we have 4 `Put(...)` implemention now...


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1542589701

   @pitrou Mind take a look at this patch when you have spare time? Since it's blocked for a long time


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1195514363


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,243 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw ParquetException("Put not implemented for " + this->descr_->ToString());
+}
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i++) {
+    // Convert to ByteArray, so we can pass to the suffix_encoder_.
+    const ByteArray value = src[i];
+    if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+      throw ParquetException("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value.ptr),
+                            static_cast<uint32_t>(value.len)};
+    uint32_t j = 0;
+    const uint32_t common_length =
+        std::min(value.len, static_cast<uint32_t>(last_value_view.length()));
+    while (j < common_length) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
+    last_value_view = view;
+    prefix_lengths[i] = j;
+    const auto suffix_length = static_cast<uint32_t>(value.len - j);
+
+    if (suffix_length == 0) {
+      suffix_encoder_.Put(&kEmpty, 1);

Review Comment:
   I believe this was necessary for some compilers. I'll comment it out temporarily to see if it's still an issue.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,243 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw ParquetException("Put not implemented for " + this->descr_->ToString());
+}
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i++) {
+    // Convert to ByteArray, so we can pass to the suffix_encoder_.
+    const ByteArray value = src[i];
+    if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+      throw ParquetException("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value.ptr),
+                            static_cast<uint32_t>(value.len)};
+    uint32_t j = 0;
+    const uint32_t common_length =
+        std::min(value.len, static_cast<uint32_t>(last_value_view.length()));
+    while (j < common_length) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
+    last_value_view = view;
+    prefix_lengths[i] = j;
+    const auto suffix_length = static_cast<uint32_t>(value.len - j);
+
+    if (suffix_length == 0) {
+      suffix_encoder_.Put(&kEmpty, 1);
+      continue;
+    }
+    const uint8_t* suffix_ptr = value.ptr + j;
+    // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+    const ByteArray suffix(suffix_length, suffix_ptr);
+    suffix_encoder_.Put(&suffix, 1);
+  }
+  prefix_length_encoder_.Put(prefix_lengths.data(), num_values);
+  last_value_ = last_value_view;
+}
+
+template <>
+void DeltaByteArrayEncoder<FLBAType>::Put(const FLBA* src, int num_values) {

Review Comment:
   Wouldn't that be a bit hard to do without making `DeltaByteArrayEncoder<FLBAType>::Put` less efficient? (I'm not sure what the overhead of getting len at every iteration is)



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,243 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw ParquetException("Put not implemented for " + this->descr_->ToString());
+}
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));

Review Comment:
   Changed. I'm not sure how would lockstep with (I assume you mean suffix encoder) would work.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3196,6 +3471,45 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+  using Base::pool_;
+
+  int Decode(ByteArray* buffer, int max_values) {
+    return GetInternal(buffer, max_values);
+  }
+  int Decode(FixedLenByteArray* buffer, int max_values) override {
+    int decoded_values_size = max_values;
+    if (MultiplyWithOverflow(decoded_values_size,
+                             descr_->type_length() * sizeof(ByteArray),

Review Comment:
   Size of a buffer that values will be decoded into.



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,293 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void CheckRoundtrip() override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType, FLBAType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 2));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_prob*/ 0));
+
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 200));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.1));
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPut) {
+  const int64_t size = 50;
+  const int32_t min_length = 0;
+  const int32_t max_length = 10;
+  const int32_t num_unique = 10;
+  const double null_probability = 0.25;
+  auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  ::arrow::random::RandomArrayGenerator rag(42);
+  auto values = rag.String(0, min_length, max_length, null_probability);
+  CheckSeed(values);
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    rag = ::arrow::random::RandomArrayGenerator(seed);
+
+    values = rag.String(size, min_length, max_length, null_probability);
+    CheckSeed(values);
+
+    values =
+        rag.BinaryWithRepeats(size, num_unique, min_length, max_length, null_probability);
+    CheckSeed(values);
+  }
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPutFixedLength) {
+  const int64_t size = 50;
+  const double null_probability = 0.25;
+  ::arrow::random::RandomArrayGenerator rag(0);
+  auto encoder = MakeTypedEncoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<FLBAType>::Accumulator acc(values->type());
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    for (auto length : {0, 10, 100, 1000}) {
+      rag = ::arrow::random::RandomArrayGenerator(seed);
+      auto values = rag.FixedSizeBinary(size, length, null_probability);
+      CheckSeed(values);
+    }
+  }
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
+  auto CheckEncode = [](std::shared_ptr<::arrow::Array> values,
+                        std::shared_ptr<::arrow::Array> prefix_lengths,
+                        std::shared_ptr<::arrow::Array> suffix_lengths,
+                        std::string_view value) {
+    auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    auto prefix_lengths_encoder =
+        MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
+    ASSERT_NO_THROW(prefix_lengths_encoder->Put(*prefix_lengths));
+    auto prefix_lengths_buf = prefix_lengths_encoder->FlushValues();
+
+    auto encoded_prefix_lengths_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
+
+    auto suffix_lengths_encoder =
+        MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
+    ASSERT_NO_THROW(suffix_lengths_encoder->Put(*suffix_lengths));
+    auto suffix_lengths_buf = suffix_lengths_encoder->FlushValues();
+    auto encoded_values_buf =
+        SliceBuffer(buf, prefix_lengths_buf->size() + suffix_lengths_buf->size());
+
+    auto encoded_prefix_length_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
+    EXPECT_TRUE(prefix_lengths_buf->Equals(*encoded_prefix_length_buf));
+    auto encoded_suffix_length_buf =
+        SliceBuffer(buf, prefix_lengths_buf->size(), suffix_lengths_buf->size());
+    EXPECT_TRUE(suffix_lengths_buf->Equals(*encoded_suffix_length_buf));
+    EXPECT_EQ(value, encoded_values_buf->ToString());
+  };
+
+  auto arrayToI32 = [](const std::shared_ptr<::arrow::Array>& lengths) {
+    std::vector<int32_t> arrays;
+    auto data_ptr = checked_cast<::arrow::Int32Array*>(lengths.get());
+    for (int i = 0; i < lengths->length(); ++i) {
+      arrays.push_back(data_ptr->GetView(i));
+    }
+    return arrays;
+  };
+
+  auto CheckDecode = [](std::shared_ptr<Buffer> buf,
+                        std::shared_ptr<::arrow::Array> values) {
+    int num_values = static_cast<int>(values->length());
+    auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(num_values, result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *upcast_result);
+  };
+
+  auto CheckEncodeDecode = [&](std::string_view values,
+                               std::shared_ptr<::arrow::Array> prefix_lengths,
+                               std::shared_ptr<::arrow::Array> suffix_lengths,
+                               std::string_view suffix_data) {
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::utf8(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::large_utf8(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::binary(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::large_binary(), values), prefix_lengths,
+                suffix_lengths, suffix_data);

Review Comment:
   Indeed! Changed.



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,293 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void CheckRoundtrip() override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType, FLBAType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 2));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_prob*/ 0));
+
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 200));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.1));
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPut) {
+  const int64_t size = 50;
+  const int32_t min_length = 0;
+  const int32_t max_length = 10;
+  const int32_t num_unique = 10;
+  const double null_probability = 0.25;
+  auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  ::arrow::random::RandomArrayGenerator rag(42);
+  auto values = rag.String(0, min_length, max_length, null_probability);
+  CheckSeed(values);
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    rag = ::arrow::random::RandomArrayGenerator(seed);
+
+    values = rag.String(size, min_length, max_length, null_probability);
+    CheckSeed(values);
+
+    values =
+        rag.BinaryWithRepeats(size, num_unique, min_length, max_length, null_probability);
+    CheckSeed(values);
+  }
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPutFixedLength) {
+  const int64_t size = 50;
+  const double null_probability = 0.25;
+  ::arrow::random::RandomArrayGenerator rag(0);
+  auto encoder = MakeTypedEncoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<FLBAType>::Accumulator acc(values->type());
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    for (auto length : {0, 10, 100, 1000}) {
+      rag = ::arrow::random::RandomArrayGenerator(seed);
+      auto values = rag.FixedSizeBinary(size, length, null_probability);
+      CheckSeed(values);
+    }
+  }
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
+  auto CheckEncode = [](std::shared_ptr<::arrow::Array> values,
+                        std::shared_ptr<::arrow::Array> prefix_lengths,
+                        std::shared_ptr<::arrow::Array> suffix_lengths,
+                        std::string_view value) {
+    auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    auto prefix_lengths_encoder =
+        MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
+    ASSERT_NO_THROW(prefix_lengths_encoder->Put(*prefix_lengths));
+    auto prefix_lengths_buf = prefix_lengths_encoder->FlushValues();
+
+    auto encoded_prefix_lengths_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
+
+    auto suffix_lengths_encoder =
+        MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
+    ASSERT_NO_THROW(suffix_lengths_encoder->Put(*suffix_lengths));
+    auto suffix_lengths_buf = suffix_lengths_encoder->FlushValues();
+    auto encoded_values_buf =
+        SliceBuffer(buf, prefix_lengths_buf->size() + suffix_lengths_buf->size());
+
+    auto encoded_prefix_length_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
+    EXPECT_TRUE(prefix_lengths_buf->Equals(*encoded_prefix_length_buf));
+    auto encoded_suffix_length_buf =
+        SliceBuffer(buf, prefix_lengths_buf->size(), suffix_lengths_buf->size());
+    EXPECT_TRUE(suffix_lengths_buf->Equals(*encoded_suffix_length_buf));
+    EXPECT_EQ(value, encoded_values_buf->ToString());
+  };
+
+  auto arrayToI32 = [](const std::shared_ptr<::arrow::Array>& lengths) {
+    std::vector<int32_t> arrays;
+    auto data_ptr = checked_cast<::arrow::Int32Array*>(lengths.get());
+    for (int i = 0; i < lengths->length(); ++i) {
+      arrays.push_back(data_ptr->GetView(i));
+    }
+    return arrays;
+  };
+
+  auto CheckDecode = [](std::shared_ptr<Buffer> buf,
+                        std::shared_ptr<::arrow::Array> values) {
+    int num_values = static_cast<int>(values->length());
+    auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(num_values, result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *upcast_result);
+  };
+
+  auto CheckEncodeDecode = [&](std::string_view values,
+                               std::shared_ptr<::arrow::Array> prefix_lengths,
+                               std::shared_ptr<::arrow::Array> suffix_lengths,
+                               std::string_view suffix_data) {
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::utf8(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::large_utf8(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::binary(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::large_binary(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+
+    auto encoded = ::arrow::ConcatenateBuffers({DeltaEncode(arrayToI32(prefix_lengths)),
+                                                DeltaEncode(arrayToI32(suffix_lengths)),
+                                                std::make_shared<Buffer>(suffix_data)})
+                       .ValueOrDie();
+
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::utf8(), values));
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_utf8(), values));
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::binary(), values));
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
+  };
+
+  {
+    auto values = R"(["axis", "axle", "babble", "babyhood"])";
+    auto prefix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([0, 2, 0, 3])");
+    auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([4, 2, 6, 5])");
+
+    constexpr std::string_view suffix_data = "axislebabbleyhood";
+    CheckEncodeDecode(values, prefix_lengths, suffix_lengths, suffix_data);
+  }
+
+  {
+    auto values = R"(["axis", "axis", "axis", "axis"])";
+    auto prefix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([0, 4, 4, 4])");
+    auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([4, 0, 0, 0])");
+
+    constexpr std::string_view suffix_data = "axis";
+    CheckEncodeDecode(values, prefix_lengths, suffix_lengths, suffix_data);
+  }
+
+  {
+    auto values = R"(["axisba", "axis", "axis", "axis"])";
+    auto prefix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([0, 4, 4, 4])");
+    auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([6, 0, 0, 0])");
+
+    constexpr std::string_view suffix_data = "axisba";
+    CheckEncodeDecode(values, prefix_lengths, suffix_lengths, suffix_data);
+  }
+
+  {
+    auto values = R"(["baaxis", "axis", "axis", "axis"])";
+    auto prefix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([0, 0, 4, 4])");
+    auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([6, 4, 0, 0])");
+
+    constexpr std::string_view suffix_data = "baaxisaxis";
+    CheckEncodeDecode(values, prefix_lengths, suffix_lengths, suffix_data);
+  }
+
+  {
+    auto values = R"(["καλημέρα", "καμηλιέρη", "καμηλιέρη", "καλημέρα"])";
+    auto prefix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([0, 5, 18, 5])");
+    auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([16, 13, 0, 11])");
+    const std::string suffix_data = "καλημέρα\xbcηλιέρη\xbbημέρα";

Review Comment:
   Indeed.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,243 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw ParquetException("Put not implemented for " + this->descr_->ToString());
+}
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i++) {
+    // Convert to ByteArray, so we can pass to the suffix_encoder_.
+    const ByteArray value = src[i];
+    if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+      throw ParquetException("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value.ptr),
+                            static_cast<uint32_t>(value.len)};

Review Comment:
   Done. Also changed at line ~3405.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,243 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw ParquetException("Put not implemented for " + this->descr_->ToString());

Review Comment:
   Indeed. Removed.



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,293 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void CheckRoundtrip() override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType, FLBAType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 2));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_prob*/ 0));
+
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 200));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.1));
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPut) {
+  const int64_t size = 50;
+  const int32_t min_length = 0;
+  const int32_t max_length = 10;
+  const int32_t num_unique = 10;
+  const double null_probability = 0.25;
+  auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());

Review Comment:
   Removing where unnecessary.
   
   `CastBinaryTypesHelper` casts (`utf8`, `large_utf8`)->`large_utf8` and (`binary`, `large_binary`)->`large_binary`. This is then used in `CheckDecode` (used in `TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut)` and `TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut)`) to be able to compare all decoded (upcast) results against the same input values.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3196,6 +3471,45 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+  using Base::pool_;
+
+  int Decode(ByteArray* buffer, int max_values) {
+    return GetInternal(buffer, max_values);
+  }
+  int Decode(FixedLenByteArray* buffer, int max_values) override {
+    int decoded_values_size = max_values;
+    if (MultiplyWithOverflow(decoded_values_size,
+                             descr_->type_length() * sizeof(ByteArray),
+                             &decoded_values_size)) {
+      throw ParquetException("excess expansion in DELTA_LENGTH_BYTE_ARRAY");
+    }
+    ArrowPoolVector<uint8_t> decode_values(decoded_values_size,
+                                           ::arrow::stl::allocator<uint8_t>(pool_));
+    auto decode_buf = reinterpret_cast<ByteArray*>(decode_values.data());

Review Comment:
   Because `GetInternal` takes `ByteArray*` (`GetInternal(ByteArray* buffer, int max_values)`). I might be missing your point :).



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3196,6 +3471,45 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+  using Base::pool_;
+
+  int Decode(ByteArray* buffer, int max_values) {
+    return GetInternal(buffer, max_values);
+  }
+  int Decode(FixedLenByteArray* buffer, int max_values) override {
+    int decoded_values_size = max_values;
+    if (MultiplyWithOverflow(decoded_values_size,
+                             descr_->type_length() * sizeof(ByteArray),
+                             &decoded_values_size)) {
+      throw ParquetException("excess expansion in DELTA_LENGTH_BYTE_ARRAY");
+    }
+    ArrowPoolVector<uint8_t> decode_values(decoded_values_size,
+                                           ::arrow::stl::allocator<uint8_t>(pool_));
+    auto decode_buf = reinterpret_cast<ByteArray*>(decode_values.data());
+
+    max_values = GetInternal(decode_buf, max_values);
+    for (int i = 0; i < max_values; i++) {
+      buffer[i].ptr = decode_buf->ptr + i * descr_->type_length();

Review Comment:
   At `3505` we decode to `ByteArray` and then copy it value by value into `FixedLenByteArray` at `3507`. We assume uniform length because original source (before encoding) was `FixedLenByteArray` (this might be assuming too much though).



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3196,6 +3471,45 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+  using Base::pool_;
+
+  int Decode(ByteArray* buffer, int max_values) {

Review Comment:
   Removing.



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1966,7 +1966,7 @@ class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
 
   void CheckRoundtrip() override {
     auto encoder =
-        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, /*xxx=*/ false, descr_.get());

Review Comment:
   Done.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3409,6 +3723,16 @@ std::unique_ptr<Encoder> MakeEncoder(Type::type type_num, Encoding::type encodin
       default:
         throw ParquetException("RLE only supports BOOLEAN");
     }
+  } else if (encoding == Encoding::DELTA_BYTE_ARRAY) {
+    switch (type_num) {
+      case Type::BYTE_ARRAY:
+        return std::make_unique<DeltaByteArrayEncoder<ByteArrayType>>(descr, pool);
+      case Type::FIXED_LEN_BYTE_ARRAY:
+        return std::make_unique<DeltaByteArrayEncoder<FLBAType>>(descr, pool);

Review Comment:
   It was motivated by https://github.com/apache/parquet-format/pull/189. I can split it out if necessary.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1196630471


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,12 +3079,243 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))),
+        prefix_lengths_(kBatchSize_, ::arrow::stl::allocator<int32_t>(pool_)) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+  ArrowPoolVector<int32_t> prefix_lengths_;
+  static constexpr int kBatchSize_ = 256;
+};
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i += kBatchSize_) {
+    const int batch_size = std::min(kBatchSize_, num_values - i);
+
+    for (int j = 0; j < batch_size; ++j) {
+      // Convert to ByteArray, so we can pass to the suffix_encoder_.
+      const ByteArray value = src[i + j];
+      if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+        throw ParquetException("Parquet cannot store strings with size 2GB or more");
+      }
+      auto view = std::string_view{value};
+
+      uint32_t k = 0;
+      const uint32_t common_length =
+          std::min(value.len, static_cast<uint32_t>(last_value_view.length()));
+      while (k < common_length) {
+        if (last_value_view[k] != view[k]) {
+          break;
+        }
+        k++;
+      }
+
+      last_value_view = view;
+      prefix_lengths_[j] = k;
+      const auto suffix_length = static_cast<uint32_t>(value.len - k);
+
+      if (suffix_length == 0) {
+        continue;
+      }
+      const uint8_t* suffix_ptr = value.ptr + k;
+      // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+      const ByteArray suffix(suffix_length, suffix_ptr);
+      suffix_encoder_.Put(&suffix, 1);
+    }
+    prefix_length_encoder_.Put(prefix_lengths_.data(), batch_size);

Review Comment:
   It's interesting that you batch the prefix encoder writes, but not the suffix encoder writes. Any particular reason?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1196621141


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,25 +1240,35 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
+template <typename DType>
+struct ArrowBinaryHelper;
+
+template <>
+struct ArrowBinaryHelper<ByteArrayType> {
   explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
     this->out = out;
     this->builder = out->builder.get();
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 this->builder->value_data_length(),
+                                                 &this->chunk_space_remaining))) {

Review Comment:
   Hmm, I see. I'm not sure `value_data_length()` could exceed 2 GB as `BinaryBuilder` should prevent against it, but we can keep this is as a precaution.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1198503212


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,229 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        auto view = visitor[i + j];
+        len = visitor.len(i + j);
+
+        uint32_t k = 0;
+        const uint32_t common_length =
+            std::min(len, static_cast<uint32_t>(last_value_view.length()));
+        while (k < common_length) {
+          if (last_value_view[k] != view[k]) {
+            break;
+          }
+          k++;
+        }
+
+        last_value_view = view;
+        prefix_lengths[j] = k;
+        const auto suffix_length = len - k;
+        const uint8_t* suffix_ptr = src[i + j].ptr + k;
+
+        // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+        const ByteArray suffix(suffix_length, suffix_ptr);
+        suffixes[j] = suffix;
+      }
+      suffix_encoder_.Put(suffixes.data(), batch_size);
+      prefix_length_encoder_.Put(prefix_lengths.data(), batch_size);
+    }
+    last_value_ = last_value_view;
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+struct ByteArrayVisitor {
+  const ByteArray* src;
+  const uint32_t length;
+
+  std::string_view operator[](int i) const {

Review Comment:
   Return a `std::string_view` is a bit wierd, I think just return a `uint8_t` or `char` is better. @pitrou 



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3409,6 +3723,16 @@ std::unique_ptr<Encoder> MakeEncoder(Type::type type_num, Encoding::type encodin
       default:
         throw ParquetException("RLE only supports BOOLEAN");
     }
+  } else if (encoding == Encoding::DELTA_BYTE_ARRAY) {
+    switch (type_num) {
+      case Type::BYTE_ARRAY:
+        return std::make_unique<DeltaByteArrayEncoder<ByteArrayType>>(descr, pool);
+      case Type::FIXED_LEN_BYTE_ARRAY:
+        return std::make_unique<DeltaByteArrayEncoder<FLBAType>>(descr, pool);

Review Comment:
   Seems that Parquet-format support applying Delta to FLBA, but I think we can split it to another patch



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,293 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void CheckRoundtrip() override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType, FLBAType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));

Review Comment:
   I've written a Random Common Prefix string generator, maybe we can use that when testing?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1205508405


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3200,6 +3461,39 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+  using Base::pool_;
+
+  int Decode(FixedLenByteArray* buffer, int max_values) override {
+    // GetInternal currently only support ByteArray.
+    std::vector<ByteArray> decode_byte_array(max_values);
+    const int decoded_values_size = GetInternal(decode_byte_array.data(), max_values);
+    const uint32_t type_length = descr_->type_length();
+
+    for (int i = 0; i < decoded_values_size; i++) {
+      if (ARROW_PREDICT_FALSE(decode_byte_array[i].len != type_length)) {
+        throw ParquetException("Fixed length byte array length mismatch");
+      }
+      buffer[i].ptr = decode_byte_array.data()->ptr + i * type_length;

Review Comment:
   This depends on the `GetInternal` implementation. Why not:
   ```suggestion
         buffer[i].ptr = decode_byte_array[i].ptr;
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1162511850


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,244 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          auto buffer_,
+          ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw ParquetException("Put not implemented for " + this->descr_->ToString());
+}
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i++) {
+    // Convert to ByteArray, so we can pass to the suffix_encoder_.
+    const ByteArray value = src[i];
+    if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+      throw ParquetException("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value.ptr),
+                            static_cast<uint32_t>(value.len)};
+    uint32_t j = 0;
+    const uint32_t common_length =
+        std::min(value.len, static_cast<uint32_t>(last_value_view.length()));
+    while (j < common_length) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
+    last_value_view = view;
+    prefix_lengths[i] = j;
+    const auto suffix_length = static_cast<uint32_t>(value.len - j);
+
+    if (suffix_length == 0) {
+      suffix_encoder_.Put(&kEmpty, 1);
+      continue;
+    }
+    const uint8_t* suffix_ptr = value.ptr + j;
+    // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+    const ByteArray suffix(suffix_length, suffix_ptr);
+    suffix_encoder_.Put(&suffix, 1);
+  }
+  prefix_length_encoder_.Put(prefix_lengths.data(), num_values);
+  last_value_ = last_value_view;
+}
+
+template <>
+void DeltaByteArrayEncoder<FLBAType>::Put(const FLBA* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+  const int32_t len = descr_->type_length();
+
+  if (ARROW_PREDICT_FALSE(len >= static_cast<int32_t>(kMaxByteArraySize))) {
+    throw Status::Invalid("Parquet cannot store strings with size 2GB or more");

Review Comment:
   Oh, I missed that one. Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1159545987


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3083,242 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, nullptr)) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          buffer_, ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);

Review Comment:
   Removed, let's see what CI says.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1268250378


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3058,235 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+  static constexpr std::string_view kEmpty = "";
+
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(descr, pool),
+        last_value_(""),
+        empty_(static_cast<uint32_t>(kEmpty.size()),
+               reinterpret_cast<const uint8_t*>(kEmpty.data())) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        const int idx = i + j;
+        auto view = visitor[idx];
+        len = visitor.len(idx);
+
+        uint32_t k = 0;
+        const uint32_t common_length =
+            std::min(len, static_cast<uint32_t>(last_value_view.length()));
+        while (k < common_length) {
+          if (last_value_view[k] != view[k]) {
+            break;
+          }
+          k++;
+        }
+
+        last_value_view = view;
+        prefix_lengths[j] = k;
+        const uint32_t suffix_length = len - k;
+        const uint8_t* suffix_ptr = src[idx].ptr + k;
+
+        // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+        const ByteArray suffix(suffix_length, suffix_ptr);
+        suffixes[j] = suffix;
+      }
+      suffix_encoder_.Put(suffixes.data(), batch_size);
+      prefix_length_encoder_.Put(prefix_lengths.data(), batch_size);
+    }
+    last_value_ = last_value_view;
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.length());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t len = src.len;
+          const uint32_t common_length = std::min(previous_len, len);

Review Comment:
   currently ok for me



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1632348068

   The CI failure on AMD64 Windows 2019 C++17 is related.
   You should also merge/rebase to fix the MinGW issues.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1623917516

   > I will take a look in depth over the weekend. Hope it can be included in the 13.0.0 release as we have been wishing it for a long time.
   
   Thanks @wgtmac! I was occupied with wooden parquet for a while but now finally have some time and am trying to figure out what's causing the issue with random generated data. I'll push what I have by tonight.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1124636904


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3265,6 +3420,14 @@ std::unique_ptr<Encoder> MakeEncoder(Type::type type_num, Encoding::type encodin
       default:
         throw ParquetException("DELTA_LENGTH_BYTE_ARRAY only supports BYTE_ARRAY");
     }
+  } else if (encoding == Encoding::DELTA_BYTE_ARRAY) {
+    switch (type_num) {
+      case Type::BYTE_ARRAY:
+        return std::make_unique<DeltaByteArrayEncoder<ByteArrayType>>(descr, pool);
+      default:
+        throw ParquetException("DELTA_BYTE_ARRAY only supports BYTE_ARRAY");

Review Comment:
   Oh indeed! I'll add it into this PR next week.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1145634331


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3062,226 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+    uint32_t previous_len = byte_width;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {

Review Comment:
   I suggest that not PREDICT here, or we can `ARROW_PREDICT_FALSE`



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3062,226 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+    uint32_t previous_len = byte_width;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = byte_width;
+          } else {
+            uint32_t j = 0;
+            while (j < previous_len) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(byte_width - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_TRUE(view.size() > kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = src.len;
+          } else {
+            uint32_t j = 0;
+            while (j < std::min(previous_len, src.len)) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+            last_value_ =

Review Comment:
   ditto



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3062,226 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+    uint32_t previous_len = byte_width;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = byte_width;
+          } else {
+            uint32_t j = 0;
+            while (j < previous_len) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(byte_width - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_TRUE(view.size() > kMaxByteArraySize)) {

Review Comment:
   The code is ok, but "with size 2GB or more" should be `">="`. And should we check it in FLBA?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3062,226 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+    uint32_t previous_len = byte_width;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = byte_width;
+          } else {
+            uint32_t j = 0;
+            while (j < previous_len) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(byte_width - j);
+            last_value_ =

Review Comment:
   Hi rok, when program is within `PutFixedLenByteArray`, `last_value_` works well, however, when `Put` is finished, and arrow array destructor, `last_value_` would point to an invalid address, causing invalid memory access.
   
   I think we can use:
   
   ```
   std::string last_value_;
   ```
   
   and in the function, we have:
   
   ```
   std::string_view last_value_view = last_value_;
   // ... change inner last_value_ to last_value_view
   
   // put it back in the end of function
   last_value_ = std::string(last_value_view);
   ```
   



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3062,226 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+    uint32_t previous_len = byte_width;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = byte_width;
+          } else {
+            uint32_t j = 0;
+            while (j < previous_len) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(byte_width - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_TRUE(view.size() > kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = src.len;
+          } else {
+            uint32_t j = 0;
+            while (j < std::min(previous_len, src.len)) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  string_view last_value_;
+};
+
+template <>
+inline void DeltaByteArrayEncoder<FLBAType>::Put(const FixedLenByteArray* src,
+                                                 int num_values) {
+  if (descr_->type_length() == 0) {
+    return;
+  }
+  for (int i = 0; i < num_values; ++i) {
+    // Write the result to the output stream
+    DCHECK(src[i].ptr != nullptr) << "Value ptr cannot be NULL";
+    PARQUET_THROW_NOT_OK(sink_.Append(src[i].ptr, descr_->type_length()));
+  }
+}
+
+template <>
+void DeltaByteArrayEncoder<FLBAType>::Put(const ::arrow::Array& values) {
+  if (!::arrow::is_fixed_size_binary(values.type_id())) {
+    throw ParquetException("Only FixedSizeBinaryArray and subclasses supported");
+  }
+  PutFixedLenByteArray(checked_cast<const ::arrow::FixedSizeBinaryArray&>(values));
+}
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const ::arrow::Array& values) {
+  AssertBaseBinary(values);
+  if (::arrow::is_binary_like(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::BinaryArray&>(values));
+  } else if (::arrow::is_large_binary_like(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::LargeBinaryArray&>(values));
+  }
+}
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values);
+
+  if (ARROW_PREDICT_TRUE(last_value_.empty())) {

Review Comment:
   not predict true here



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3062,226 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+    uint32_t previous_len = byte_width;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = byte_width;
+          } else {
+            uint32_t j = 0;
+            while (j < previous_len) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(byte_width - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_TRUE(view.size() > kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = src.len;
+          } else {
+            uint32_t j = 0;
+            while (j < std::min(previous_len, src.len)) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  string_view last_value_;
+};
+
+template <>
+inline void DeltaByteArrayEncoder<FLBAType>::Put(const FixedLenByteArray* src,
+                                                 int num_values) {
+  if (descr_->type_length() == 0) {
+    return;
+  }
+  for (int i = 0; i < num_values; ++i) {
+    // Write the result to the output stream
+    DCHECK(src[i].ptr != nullptr) << "Value ptr cannot be NULL";
+    PARQUET_THROW_NOT_OK(sink_.Append(src[i].ptr, descr_->type_length()));
+  }
+}
+
+template <>
+void DeltaByteArrayEncoder<FLBAType>::Put(const ::arrow::Array& values) {
+  if (!::arrow::is_fixed_size_binary(values.type_id())) {
+    throw ParquetException("Only FixedSizeBinaryArray and subclasses supported");
+  }
+  PutFixedLenByteArray(checked_cast<const ::arrow::FixedSizeBinaryArray&>(values));
+}
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const ::arrow::Array& values) {
+  AssertBaseBinary(values);
+  if (::arrow::is_binary_like(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::BinaryArray&>(values));
+  } else if (::arrow::is_large_binary_like(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::LargeBinaryArray&>(values));
+  }
+}
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values);
+
+  if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+    last_value_ = string_view{reinterpret_cast<const char*>(src[0].ptr), src[0].len};
+    suffix_encoder_.Put(&src[0], 1);
+    prefix_lengths[0] = 0;
+  }
+
+  for (int32_t i = 1; i < num_values; i++) {

Review Comment:
   a bit confusing here, if `last_value_` is not empty, the first value would be ignored.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -1275,6 +1275,35 @@ struct ArrowBinaryHelper {
   int64_t chunk_space_remaining;
 };
 
+struct ArrowFLBAHelper {
+  explicit ArrowFLBAHelper(::arrow::FixedSizeBinaryBuilder* builder) {
+    this->builder = builder;
+    this->chunk_space_remaining =
+        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+  }
+
+  Status PushChunk() {
+    std::shared_ptr<::arrow::Array> result;
+    RETURN_NOT_OK(builder->Finish(&result));
+    chunks.push_back(result);

Review Comment:
   `chunks.push_back(std::move(result));`



##########
cpp/src/parquet/encoding.cc:
##########
@@ -2900,8 +2929,222 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+ public:
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr, MemoryPool* pool)
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY,
+                    pool = ::arrow::default_memory_pool()),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+
+    // TODO(rok): optimize using ArrowPoolVector<int32_t> prefix_lengths(num_values);
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          uint32_t previous_len = 0;
+          const ByteArray src{view};

Review Comment:
   okay, you can leave a comment for it, I think it's ok but a bit confusing for code reader, because FLBA will change it to another type here.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3062,226 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+    uint32_t previous_len = byte_width;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = byte_width;
+          } else {
+            uint32_t j = 0;
+            while (j < previous_len) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(byte_width - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_TRUE(view.size() > kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = src.len;
+          } else {
+            uint32_t j = 0;
+            while (j < std::min(previous_len, src.len)) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  string_view last_value_;
+};
+
+template <>
+inline void DeltaByteArrayEncoder<FLBAType>::Put(const FixedLenByteArray* src,
+                                                 int num_values) {
+  if (descr_->type_length() == 0) {
+    return;
+  }
+  for (int i = 0; i < num_values; ++i) {
+    // Write the result to the output stream
+    DCHECK(src[i].ptr != nullptr) << "Value ptr cannot be NULL";
+    PARQUET_THROW_NOT_OK(sink_.Append(src[i].ptr, descr_->type_length()));
+  }
+}
+
+template <>
+void DeltaByteArrayEncoder<FLBAType>::Put(const ::arrow::Array& values) {
+  if (!::arrow::is_fixed_size_binary(values.type_id())) {
+    throw ParquetException("Only FixedSizeBinaryArray and subclasses supported");
+  }
+  PutFixedLenByteArray(checked_cast<const ::arrow::FixedSizeBinaryArray&>(values));
+}
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const ::arrow::Array& values) {
+  AssertBaseBinary(values);
+  if (::arrow::is_binary_like(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::BinaryArray&>(values));
+  } else if (::arrow::is_large_binary_like(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::LargeBinaryArray&>(values));
+  }
+}
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values);
+
+  if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+    last_value_ = string_view{reinterpret_cast<const char*>(src[0].ptr), src[0].len};
+    suffix_encoder_.Put(&src[0], 1);
+    prefix_lengths[0] = 0;
+  }
+
+  for (int32_t i = 1; i < num_values; i++) {
+    auto prefix = string_view{reinterpret_cast<const char*>(src[i].ptr), src[i].len};
+
+    uint32_t j = 0;
+    while (j < std::min(src[i - 1].len, src[i].len)) {
+      if (last_value_[j] != prefix[j]) {
+        break;
+      }
+      j++;
+    }
+
+    prefix_lengths[i] = j;
+    const uint8_t* suffix_ptr = src[i].ptr + j;
+    const uint32_t suffix_length = static_cast<uint32_t>(src[i].len - j);
+    last_value_ = string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};

Review Comment:
   ditto



##########
cpp/src/parquet/encoding.cc:
##########
@@ -1268,6 +1268,35 @@ struct ArrowBinaryHelper {
   int64_t chunk_space_remaining;
 };
 
+struct ArrowFLBAHelper {
+  explicit ArrowFLBAHelper(::arrow::FixedSizeBinaryBuilder* builder) {
+    this->builder = builder;
+    this->chunk_space_remaining =
+        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+  }
+
+  Status PushChunk() {
+    std::shared_ptr<::arrow::Array> result;
+    RETURN_NOT_OK(builder->Finish(&result));
+    chunks.push_back(result);
+    chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
+    return Status::OK();
+  }
+
+  bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
+
+  Status Append(const uint8_t* data, int32_t length) {
+    chunk_space_remaining -= length;

Review Comment:
   no, I just want a Debug time check here. No checking is ok for me



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3062,226 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+    uint32_t previous_len = byte_width;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = byte_width;
+          } else {
+            uint32_t j = 0;
+            while (j < previous_len) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(byte_width - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_TRUE(view.size() > kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = src.len;
+          } else {
+            uint32_t j = 0;
+            while (j < std::min(previous_len, src.len)) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  string_view last_value_;
+};
+
+template <>
+inline void DeltaByteArrayEncoder<FLBAType>::Put(const FixedLenByteArray* src,
+                                                 int num_values) {
+  if (descr_->type_length() == 0) {
+    return;
+  }
+  for (int i = 0; i < num_values; ++i) {
+    // Write the result to the output stream
+    DCHECK(src[i].ptr != nullptr) << "Value ptr cannot be NULL";
+    PARQUET_THROW_NOT_OK(sink_.Append(src[i].ptr, descr_->type_length()));
+  }
+}
+
+template <>
+void DeltaByteArrayEncoder<FLBAType>::Put(const ::arrow::Array& values) {
+  if (!::arrow::is_fixed_size_binary(values.type_id())) {
+    throw ParquetException("Only FixedSizeBinaryArray and subclasses supported");
+  }
+  PutFixedLenByteArray(checked_cast<const ::arrow::FixedSizeBinaryArray&>(values));
+}
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const ::arrow::Array& values) {
+  AssertBaseBinary(values);
+  if (::arrow::is_binary_like(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::BinaryArray&>(values));
+  } else if (::arrow::is_large_binary_like(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::LargeBinaryArray&>(values));
+  }
+}
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values);

Review Comment:
   ```
   ArrowPoolVector<int32_t> prefix_lengths(num_values, pool_);
   ```
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1149515227


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,180 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = j;

Review Comment:
   ```suggestion
             previous_len = src.len;
   ```



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,180 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;

Review Comment:
   ```suggestion
       uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
   ```



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,180 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = j;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          const uint8_t* suffix_ptr = src.ptr + j;
+          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+          last_value_view =
+              string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+          // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  int i = 0;
+  while (i < num_values) {
+    // Convert to ByteArray so we can pass to the suffix_encoder_.
+    auto value = reinterpret_cast<const ByteArray*>(&src[i]);
+    if (ARROW_PREDICT_FALSE(value->len >= kMaxByteArraySize)) {
+      throw Status::Invalid("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value->ptr), value->len};
+    uint32_t j = 0;
+    while (j < std::min(value->len, static_cast<uint32_t>(last_value_view.length()))) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
+    prefix_lengths[i] = j;
+    const uint8_t* suffix_ptr = value->ptr + j;
+    const uint32_t suffix_length = static_cast<uint32_t>(value->len - j);
+    last_value_view =
+        string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+    // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+    const ByteArray suffix(suffix_length, suffix_ptr);
+    suffix_encoder_.Put(&suffix, 1);
+    i++;
+  }
+  prefix_length_encoder_.Put(prefix_lengths.data(), num_values);
+  last_value_ = last_value_view;
+}
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const ::arrow::Array& values) {
+  if (::arrow::is_binary_like(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::BinaryArray&>(values));
+  } else if (::arrow::is_large_binary_like(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::LargeBinaryArray&>(values));
+  } else if (::arrow::is_fixed_size_binary(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::FixedSizeBinaryArray&>(values));
+  } else {
+    throw ParquetException("Only BaseBinaryArray and subclasses supported");
+  }
+}
+
+template <typename DType>
+std::shared_ptr<Buffer> DeltaByteArrayEncoder<DType>::FlushValues() {
+  PARQUET_THROW_NOT_OK(sink_.Resize(EstimatedDataEncodedSize(), false));
+
+  std::shared_ptr<Buffer> prefix_lengths = prefix_length_encoder_.FlushValues();
+  PARQUET_THROW_NOT_OK(sink_.Append(prefix_lengths->data(), prefix_lengths->size()));
+
+  std::shared_ptr<Buffer> suffixes = suffix_encoder_.FlushValues();
+  PARQUET_THROW_NOT_OK(sink_.Append(suffixes->data(), suffixes->size()));
+
+  std::shared_ptr<Buffer> buffer;
+  PARQUET_THROW_NOT_OK(sink_.Finish(&buffer, true));
+  return buffer;

Review Comment:
   Should clear context here.
   
   ```suggestion
     last_value_.clear();
     return buffer;
   ```



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,180 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = j;

Review Comment:
   Or we can discard `previous_len`, just use `last_value_view`



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,159 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  virtual void CheckRoundtrip() {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits, int64_t valid_bits_offset) {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+// TODO: add FLBAType and Decimal type tests
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 2));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_prob*/ 0));
+
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 200));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.1));
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPut) {
+  const int64_t size = 50;
+  const int32_t min_length = 0;
+  const int32_t max_length = 10;
+  const int32_t num_unique = 10;
+  const double null_probability = 0.25;
+  auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  ::arrow::random::RandomArrayGenerator rag(42);
+  auto values = rag.String(0, min_length, max_length, null_probability);
+  CheckSeed(values);
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    rag = ::arrow::random::RandomArrayGenerator(seed);
+
+    values = rag.String(size, min_length, max_length, null_probability);
+    CheckSeed(values);
+
+    values =
+        rag.BinaryWithRepeats(size, num_unique, min_length, max_length, null_probability);
+    CheckSeed(values);
+  }
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPutFixedLength) {
+  const int64_t size = 50;
+  const double null_probability = 0.25;
+  ::arrow::random::RandomArrayGenerator rag(0);
+  auto encoder = MakeTypedEncoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<FLBAType>::Accumulator acc(values->type());
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    for (auto length : {0, 10, 100, 1000}) {
+      rag = ::arrow::random::RandomArrayGenerator(seed);
+      auto values = rag.FixedSizeBinary(size, length, null_probability);
+      CheckSeed(values);
+    }
+  }
+}
+

Review Comment:
   Should add a test for common prefix, seems that arrow generate didn't tests that:
   
   ```suggestion
   TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
     auto CheckEncode = [](std::shared_ptr<::arrow::Array> values,
                           std::shared_ptr<::arrow::Array> prefix_lengths,
                           std::shared_ptr<::arrow::Array> suffix_lengths, std::string_view value)  {
       auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
       ASSERT_NO_THROW(encoder->Put(*values));
       auto buf = encoder->FlushValues();
   
       auto prefix_lengths_encoder = MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
       ASSERT_NO_THROW(prefix_lengths_encoder->Put(*prefix_lengths));
       auto prefix_lengths_buf = prefix_lengths_encoder->FlushValues();
   
       auto encoded_prefix_lengths_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
   
       auto suffix_lengths_encoder = MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
       ASSERT_NO_THROW(suffix_lengths_encoder->Put(*suffix_lengths));
       auto suffix_lengths_buf = suffix_lengths_encoder->FlushValues();
   
       auto encoded_values_buf = SliceBuffer(buf, prefix_lengths_buf->size() + suffix_lengths_buf->size());
   
       auto encoded_prefix_length_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
       EXPECT_TRUE(prefix_lengths_buf->Equals(*encoded_prefix_length_buf));
       auto encoded_suffix_length_buf = SliceBuffer(buf, prefix_lengths_buf->size(), suffix_lengths_buf->size());
       EXPECT_TRUE(suffix_lengths_buf->Equals(*encoded_suffix_length_buf));
       EXPECT_EQ(value, encoded_values_buf->ToString());
     };
   
     auto values = R"(["axis", "axle", "babble", "babyhood"])";
     auto prefix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([0, 2, 0, 3])");
     auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([4, 2, 6, 5])");
   
     CheckEncode(::arrow::ArrayFromJSON(::arrow::utf8(), values), prefix_lengths, suffix_lengths, "axislebabbleyhood");
     CheckEncode(::arrow::ArrayFromJSON(::arrow::large_utf8(), values), prefix_lengths, suffix_lengths, "axislebabbleyhood");
     CheckEncode(::arrow::ArrayFromJSON(::arrow::binary(), values), prefix_lengths, suffix_lengths, "axislebabbleyhood");
     CheckEncode(::arrow::ArrayFromJSON(::arrow::large_binary(), values), prefix_lengths, suffix_lengths, "axislebabbleyhood");
   }
   ```
   
   This code is modified from `DeltaLengthByteArrayEncodingAdHoc.ArrowDirectPut`, you can just go ahead and add `CheckDecode`, and add more cases



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1182935442


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1275,6 +1287,40 @@ struct ArrowBinaryHelper {
   int64_t chunk_space_remaining;
 };
 
+template <>
+struct ArrowBinaryHelper<FLBAType> {
+  explicit ArrowBinaryHelper(EncodingTraits<FLBAType>::Accumulator* builder) {
+    this->builder = builder;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 this->builder->value_data_length(),
+                                                 &this->chunk_space_remaining))) {
+      throw ParquetException("excess expansion in DELTA_LENGTH_BYTE_ARRAY");

Review Comment:
   Changed to:
   ```cpp
   throw ParquetException("excess expansion in ArrowBinaryHelper<FLBAType>");
   ```



##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,25 +1240,35 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
+template <typename DType>
+struct ArrowBinaryHelper;
+
+template <>
+struct ArrowBinaryHelper<ByteArrayType> {
   explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
     this->out = out;
     this->builder = out->builder.get();
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 this->builder->value_data_length(),
+                                                 &this->chunk_space_remaining))) {
+      throw ParquetException("excess expansion in DELTA_LENGTH_BYTE_ARRAY");

Review Comment:
   Changed to:
   ```cpp
   throw ParquetException("excess expansion in ArrowBinaryHelper<ByteArrayType>");
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1667638555

   @pitrou now that 13.0.0 is (mostly) done, could we finalize this so we have some safety buffer until 14.0.0?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] anjakefala commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "anjakefala (via GitHub)" <gi...@apache.org>.
anjakefala commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1686947498

   Congratulations, @rok!!!
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer [arrow]

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1386818878


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1335,13 +1420,7 @@ class PlainByteArrayDecoder : public PlainDecoder<ByteArrayType>,
           if (ARROW_PREDICT_FALSE(len_ < increment)) {
             ParquetException::EofException();
           }
-          if (ARROW_PREDICT_FALSE(!helper.CanFit(value_len))) {
-            // This element would exceed the capacity of a chunk
-            RETURN_NOT_OK(helper.PushChunk());
-            RETURN_NOT_OK(helper.builder->Reserve(num_values - i));
-            RETURN_NOT_OK(helper.builder->ReserveData(
-                std::min<int64_t>(len_, helper.chunk_space_remaining)));

Review Comment:
   I don't understand: are you commented on the removed code? I'd rather not try to understand code that was removed months ago...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1297650296


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -874,7 +874,7 @@ std::shared_ptr<::arrow::Array> EncodingAdHocTyped<FLBAType>::GetValues(int seed
 }
 
 using EncodingAdHocTypedCases =
-    ::testing::Types<BooleanType, Int32Type, Int64Type, FloatType, DoubleType, FLBAType>;
+    ::testing::Types<BooleanType, Int32Type, Int64Type, FloatType, DoubleType>;

Review Comment:
   That's a good point. I reverted the `EncodingTraits<FLBAType>` change and expanded `ArrowBinaryHelper` to support `FLBA`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1295943448


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,230 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        auto view = visitor[i + j];
+        len = visitor.len(i + j);
+
+        uint32_t k = 0;
+        const uint32_t common_length =
+            std::min(len, static_cast<uint32_t>(last_value_view.length()));
+        while (k < common_length) {
+          if (last_value_view[k] != view[k]) {
+            break;
+          }
+          k++;
+        }
+
+        last_value_view = view;
+        prefix_lengths[j] = k;
+        const uint32_t suffix_length = len - k;
+        const uint8_t* suffix_ptr = src[i + j].ptr + k;
+
+        // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+        const ByteArray suffix(suffix_length, suffix_ptr);
+        suffixes[j] = suffix;
+      }
+      suffix_encoder_.Put(suffixes.data(), batch_size);
+      prefix_length_encoder_.Put(prefix_lengths.data(), batch_size);
+    }
+    last_value_ = last_value_view;
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.length());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t len = src.len;
+          const uint32_t common_length = std::min(previous_len, len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+struct ByteArrayVisitor {
+  const ByteArray* src;
+  const uint32_t type_length;

Review Comment:
   See comment just above :-)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1295879261


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -874,7 +874,7 @@ std::shared_ptr<::arrow::Array> EncodingAdHocTyped<FLBAType>::GetValues(int seed
 }
 
 using EncodingAdHocTypedCases =
-    ::testing::Types<BooleanType, Int32Type, Int64Type, FloatType, DoubleType, FLBAType>;
+    ::testing::Types<BooleanType, Int32Type, Int64Type, FloatType, DoubleType>;

Review Comment:
   Well... the thing is, the change brings pointless complication from the user's POV, isn't it? FixedSizeBinary arrays don't need to be broken down in multiple chunks because they have no limitation on data size.
   
   If it's purely to ease templating, why don't you use a private templated helper? You even already have `ArrowBinaryHelper` which would perhaps be expanded or adapted.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1299823354


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,43 +1240,93 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
-  explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
-    this->out = out;
-    this->builder = out->builder.get();
-    this->chunk_space_remaining =
-        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+template <typename DType, typename Enable = void>
+struct ArrowBinaryHelper;
+
+template <typename DType>
+struct ArrowBinaryHelper<DType,
+                         std::enable_if_t<std::is_same_v<DType, ByteArrayType>, void>> {
+  explicit ArrowBinaryHelper(typename EncodingTraits<DType>::Accumulator* acc) {
+    builder = acc->builder.get();
+    chunks = &acc->chunks;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 builder->value_data_length(),
+                                                 &chunk_space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<DType>");
+    }
   }
 
   Status PushChunk() {
     std::shared_ptr<::arrow::Array> result;
     RETURN_NOT_OK(builder->Finish(&result));
-    out->chunks.push_back(result);
+    chunks->push_back(result);
     chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
     return Status::OK();
   }
 
   bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
 
   void UnsafeAppend(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
     builder->UnsafeAppend(data, length);
   }
 
   void UnsafeAppendNull() { builder->UnsafeAppendNull(); }
 
   Status Append(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
     return builder->Append(data, length);
   }
 
   Status AppendNull() { return builder->AppendNull(); }
 
-  typename EncodingTraits<ByteArrayType>::Accumulator* out;
-  ::arrow::BinaryBuilder* builder;
+  typename EncodingTraits<DType>::BuilderType* builder;
+  std::vector<std::shared_ptr<::arrow::Array>>* chunks;
   int64_t chunk_space_remaining;
 };
 
+template <typename DType>
+struct ArrowBinaryHelper<DType, std::enable_if_t<std::is_same_v<DType, FLBAType>, void>> {
+  explicit ArrowBinaryHelper(typename EncodingTraits<DType>::Accumulator* acc) {
+    builder = acc;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 builder->value_data_length(),
+                                                 &space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<DType>");
+    }
+  }
+
+  Status PushChunk() {
+    std::shared_ptr<::arrow::Array> result;
+    RETURN_NOT_OK(builder->Finish(&result));

Review Comment:
   I'm reading through this again and I don't understand what you're doing. Are you throwing away the `result`?
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1300188563


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,40 +1240,54 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
+template <typename DType>
 struct ArrowBinaryHelper {
-  explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
-    this->out = out;
-    this->builder = out->builder.get();
-    this->chunk_space_remaining =
-        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+  explicit ArrowBinaryHelper(typename EncodingTraits<DType>::Accumulator* acc) {
+    if constexpr (std::is_same_v<DType, ByteArrayType>) {
+      builder = acc->builder.get();
+      chunks = &acc->chunks;
+    } else {
+      builder = acc;
+    }
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 builder->value_data_length(),
+                                                 &chunk_space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<DType>");
+    }
   }
 
   Status PushChunk() {
     std::shared_ptr<::arrow::Array> result;
     RETURN_NOT_OK(builder->Finish(&result));
-    out->chunks.push_back(result);
+    chunks->push_back(result);
     chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
     return Status::OK();
   }
 
   bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
 
   void UnsafeAppend(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
     builder->UnsafeAppend(data, length);
   }
 
   void UnsafeAppendNull() { builder->UnsafeAppendNull(); }
 
   Status Append(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
-    return builder->Append(data, length);
+    if constexpr (std::is_same_v<DType, FLBAType>) {
+      return builder->Append(data);
+    } else {
+      return builder->Append(data, length);
+    }
   }
 
   Status AppendNull() { return builder->AppendNull(); }
 
-  typename EncodingTraits<ByteArrayType>::Accumulator* out;
-  ::arrow::BinaryBuilder* builder;
+  typename EncodingTraits<DType>::BuilderType* builder;
+  std::vector<std::shared_ptr<::arrow::Array>>* chunks;

Review Comment:
   How about we raise here until the fix?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1198509202


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,229 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        auto view = visitor[i + j];
+        len = visitor.len(i + j);
+
+        uint32_t k = 0;
+        const uint32_t common_length =
+            std::min(len, static_cast<uint32_t>(last_value_view.length()));
+        while (k < common_length) {
+          if (last_value_view[k] != view[k]) {
+            break;
+          }
+          k++;
+        }
+
+        last_value_view = view;
+        prefix_lengths[j] = k;
+        const auto suffix_length = len - k;
+        const uint8_t* suffix_ptr = src[i + j].ptr + k;
+
+        // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+        const ByteArray suffix(suffix_length, suffix_ptr);
+        suffixes[j] = suffix;
+      }
+      suffix_encoder_.Put(suffixes.data(), batch_size);
+      prefix_length_encoder_.Put(prefix_lengths.data(), batch_size);
+    }
+    last_value_ = last_value_view;
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+struct ByteArrayVisitor {
+  const ByteArray* src;
+  const uint32_t length;
+
+  std::string_view operator[](int i) const {
+    if (ARROW_PREDICT_FALSE(src[i].len >= kMaxByteArraySize)) {
+      throw ParquetException("Parquet cannot store strings with size 2GB or more");
+    }
+    return std::string_view{src[i]};
+  }
+
+  const uint32_t len(int i) const { return src[i].len; }
+};
+
+struct FLBAVisitor {
+  const FLBA* src;
+  const uint32_t length;
+
+  std::string_view operator[](int i) const {
+    return std::string_view{reinterpret_cast<const char*>(src[i].ptr)};

Review Comment:
   You should explicit set the length for flba, otherwise I think it will be regard as C style string and truncate the content after `\0`.
   
   Here maybe we need to acquire length of `FLBA`, so I guess maybe we need an extra argument?
   
   ```c++
     std::string_view operator[](int i) const {
       return std::string_view{reinterpret_cast<const char*>(src[i].ptr, flba.length)};
     }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1197174534


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,12 +3079,243 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))),
+        prefix_lengths_(kBatchSize_, ::arrow::stl::allocator<int32_t>(pool_)) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+  ArrowPoolVector<int32_t> prefix_lengths_;
+  static constexpr int kBatchSize_ = 256;
+};
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i += kBatchSize_) {
+    const int batch_size = std::min(kBatchSize_, num_values - i);
+
+    for (int j = 0; j < batch_size; ++j) {
+      // Convert to ByteArray, so we can pass to the suffix_encoder_.
+      const ByteArray value = src[i + j];
+      if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+        throw ParquetException("Parquet cannot store strings with size 2GB or more");
+      }
+      auto view = std::string_view{value};
+
+      uint32_t k = 0;
+      const uint32_t common_length =
+          std::min(value.len, static_cast<uint32_t>(last_value_view.length()));
+      while (k < common_length) {
+        if (last_value_view[k] != view[k]) {
+          break;
+        }
+        k++;
+      }
+
+      last_value_view = view;
+      prefix_lengths_[j] = k;
+      const auto suffix_length = static_cast<uint32_t>(value.len - k);
+
+      if (suffix_length == 0) {
+        continue;
+      }
+      const uint8_t* suffix_ptr = value.ptr + k;
+      // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+      const ByteArray suffix(suffix_length, suffix_ptr);
+      suffix_encoder_.Put(&suffix, 1);
+    }
+    prefix_length_encoder_.Put(prefix_lengths_.data(), batch_size);

Review Comment:
   Prefix batches are of predictable sizes, while suffix batches would not be. We could guess at sizes, but I'm not sure how productive that would be.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1205509750


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1956,5 +1955,299 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  //  void InitData(int nvalues) {

Review Comment:
   Why the commented out code?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1501954385

   If there are no objections I'd like to merge this before 12.0.0 window closes tomorrow (Tuesday).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1157757180


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,285 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  virtual void CheckRoundtrip() {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits, int64_t valid_bits_offset) {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+// TODO: add FLBAType and Decimal type tests
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 2));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_prob*/ 0));
+
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 200));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.1));
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPut) {
+  const int64_t size = 50;
+  const int32_t min_length = 0;
+  const int32_t max_length = 10;
+  const int32_t num_unique = 10;
+  const double null_probability = 0.25;
+  auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  ::arrow::random::RandomArrayGenerator rag(42);
+  auto values = rag.String(0, min_length, max_length, null_probability);
+  CheckSeed(values);
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    rag = ::arrow::random::RandomArrayGenerator(seed);
+
+    values = rag.String(size, min_length, max_length, null_probability);
+    CheckSeed(values);
+
+    values =
+        rag.BinaryWithRepeats(size, num_unique, min_length, max_length, null_probability);
+    CheckSeed(values);
+  }
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPutFixedLength) {
+  const int64_t size = 50;
+  const double null_probability = 0.25;
+  ::arrow::random::RandomArrayGenerator rag(0);
+  auto encoder = MakeTypedEncoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<FLBAType>::Accumulator acc(values->type());
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    for (auto length : {0, 10, 100, 1000}) {
+      rag = ::arrow::random::RandomArrayGenerator(seed);
+      auto values = rag.FixedSizeBinary(size, length, null_probability);
+      CheckSeed(values);
+    }
+  }
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
+  auto CheckEncode = [](std::shared_ptr<::arrow::Array> values,
+                        std::shared_ptr<::arrow::Array> prefix_lengths,
+                        std::shared_ptr<::arrow::Array> suffix_lengths,
+                        std::string_view value) {
+    auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    auto prefix_lengths_encoder =
+        MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
+    ASSERT_NO_THROW(prefix_lengths_encoder->Put(*prefix_lengths));
+    auto prefix_lengths_buf = prefix_lengths_encoder->FlushValues();
+
+    auto encoded_prefix_lengths_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
+
+    auto suffix_lengths_encoder =
+        MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
+    ASSERT_NO_THROW(suffix_lengths_encoder->Put(*suffix_lengths));
+    auto suffix_lengths_buf = suffix_lengths_encoder->FlushValues();
+    auto encoded_values_buf =
+        SliceBuffer(buf, prefix_lengths_buf->size() + suffix_lengths_buf->size());
+
+    auto encoded_prefix_length_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
+    EXPECT_TRUE(prefix_lengths_buf->Equals(*encoded_prefix_length_buf));
+    auto encoded_suffix_length_buf =
+        SliceBuffer(buf, prefix_lengths_buf->size(), suffix_lengths_buf->size());
+    EXPECT_TRUE(suffix_lengths_buf->Equals(*encoded_suffix_length_buf));
+    EXPECT_EQ(value, encoded_values_buf->ToString());
+  };
+
+  auto arrayToI32 = [](const std::shared_ptr<::arrow::Array>& lengths) {
+    std::vector<int32_t> arrays;
+    auto data_ptr = checked_cast<::arrow::Int32Array*>(lengths.get());
+    for (int i = 0; i < lengths->length(); ++i) {
+      arrays.push_back(data_ptr->GetView(i));
+    }
+    return arrays;
+  };
+
+  auto CheckDecode = [](std::shared_ptr<Buffer> buf,
+                        std::shared_ptr<::arrow::Array> values) {
+    int num_values = static_cast<int>(values->length());
+    auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(num_values, result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *upcast_result);
+  };
+
+  auto checkEncodeDecode = [&](std::string_view values,
+                               std::shared_ptr<::arrow::Array> prefix_lengths,
+                               std::shared_ptr<::arrow::Array> suffix_lengths,
+                               std::string_view suffix_data) {
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::utf8(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::large_utf8(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::binary(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::large_binary(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+
+    auto encoded = ::arrow::ConcatenateBuffers({DeltaEncode(arrayToI32(prefix_lengths)),
+                                                DeltaEncode(arrayToI32(suffix_lengths)),
+                                                std::make_shared<Buffer>(suffix_data)})
+                       .ValueOrDie();
+
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::utf8(), values));
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_utf8(), values));
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::binary(), values));
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
+  };
+
+  {
+    auto values = R"(["axis", "axle", "babble", "babyhood"])";

Review Comment:
   Ah, actually this passes, so I think we're good:
   ```
     {
       auto values = R"(["καλημέρα", "καμηλιέρη", "καμηλιέρη", "καλημέρα"])";
       auto prefix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([0, 5, 18, 5])");
       auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([16, 13, 0, 11])");
       const std::string suffix_data = "καλημέρα\xbcηλιέρη\xbbημέρα";
       CheckEncodeDecode(values, prefix_lengths, suffix_lengths, suffix_data);
     }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1195344198


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,25 +1240,35 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
+template <typename DType>
+struct ArrowBinaryHelper;
+
+template <>
+struct ArrowBinaryHelper<ByteArrayType> {
   explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
     this->out = out;
     this->builder = out->builder.get();
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 this->builder->value_data_length(),
+                                                 &this->chunk_space_remaining))) {

Review Comment:
   I'm curious, in which circumstances can it overflow? These are all 64-bit integers.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,243 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw ParquetException("Put not implemented for " + this->descr_->ToString());
+}
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i++) {
+    // Convert to ByteArray, so we can pass to the suffix_encoder_.
+    const ByteArray value = src[i];
+    if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+      throw ParquetException("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value.ptr),
+                            static_cast<uint32_t>(value.len)};
+    uint32_t j = 0;
+    const uint32_t common_length =
+        std::min(value.len, static_cast<uint32_t>(last_value_view.length()));
+    while (j < common_length) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
+    last_value_view = view;
+    prefix_lengths[i] = j;
+    const auto suffix_length = static_cast<uint32_t>(value.len - j);
+
+    if (suffix_length == 0) {
+      suffix_encoder_.Put(&kEmpty, 1);

Review Comment:
   Why is this necessary exactly? I think `suffix` below would be valid, did you get a failure somewhere?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,243 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw ParquetException("Put not implemented for " + this->descr_->ToString());
+}
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i++) {
+    // Convert to ByteArray, so we can pass to the suffix_encoder_.
+    const ByteArray value = src[i];
+    if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+      throw ParquetException("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value.ptr),
+                            static_cast<uint32_t>(value.len)};

Review Comment:
   Ok, let's instead add a conversion operator to `ByteArray`?
   ```c++
   struct ByteArray {
     explicit operator std::string_view() { return std::string_view{reinterpret_cast<const char*>(ptr), len}; }
   ...
   ```



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,243 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw ParquetException("Put not implemented for " + this->descr_->ToString());
+}
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));

Review Comment:
   This will potentially allocate and deallocate a large array each time `Put` is called.
   
   It might be more efficient to work by chunks by creating a small static pool vector in `DeltaByteArrayEncoder`.
   
   (even better might be to chunk in lockstep with the prefix encoder, but more involved)



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3196,6 +3471,45 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+  using Base::pool_;
+
+  int Decode(ByteArray* buffer, int max_values) {
+    return GetInternal(buffer, max_values);
+  }
+  int Decode(FixedLenByteArray* buffer, int max_values) override {
+    int decoded_values_size = max_values;
+    if (MultiplyWithOverflow(decoded_values_size,
+                             descr_->type_length() * sizeof(ByteArray),
+                             &decoded_values_size)) {
+      throw ParquetException("excess expansion in DELTA_LENGTH_BYTE_ARRAY");
+    }
+    ArrowPoolVector<uint8_t> decode_values(decoded_values_size,
+                                           ::arrow::stl::allocator<uint8_t>(pool_));
+    auto decode_buf = reinterpret_cast<ByteArray*>(decode_values.data());

Review Comment:
   Why not use a `vector<ByteArray>` directly?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3409,6 +3723,16 @@ std::unique_ptr<Encoder> MakeEncoder(Type::type type_num, Encoding::type encodin
       default:
         throw ParquetException("RLE only supports BOOLEAN");
     }
+  } else if (encoding == Encoding::DELTA_BYTE_ARRAY) {
+    switch (type_num) {
+      case Type::BYTE_ARRAY:
+        return std::make_unique<DeltaByteArrayEncoder<ByteArrayType>>(descr, pool);
+      case Type::FIXED_LEN_BYTE_ARRAY:
+        return std::make_unique<DeltaByteArrayEncoder<FLBAType>>(descr, pool);

Review Comment:
   I'm not sure why you decided to add FLBA support. I would probably keep things simple, though why not?



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,293 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void CheckRoundtrip() override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType, FLBAType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));

Review Comment:
   The problem with purely random data is that you statistically have almost no prefixes.
   I think you want to override `InitData` and generate some more interesting data, for example by having a random generator for prefix and suffix lengths.



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,293 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void CheckRoundtrip() override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType, FLBAType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 2));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_prob*/ 0));
+
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 200));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.1));
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPut) {
+  const int64_t size = 50;
+  const int32_t min_length = 0;
+  const int32_t max_length = 10;
+  const int32_t num_unique = 10;
+  const double null_probability = 0.25;
+  auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  ::arrow::random::RandomArrayGenerator rag(42);
+  auto values = rag.String(0, min_length, max_length, null_probability);

Review Comment:
   Same remark here: pure random generation will not give us very interesting data.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,243 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw ParquetException("Put not implemented for " + this->descr_->ToString());

Review Comment:
   Why is this necessary? We should never instantiate a `DeltaByteArrayEncoder` for an unsupported DType.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3196,6 +3471,45 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+  using Base::pool_;
+
+  int Decode(ByteArray* buffer, int max_values) {
+    return GetInternal(buffer, max_values);
+  }
+  int Decode(FixedLenByteArray* buffer, int max_values) override {
+    int decoded_values_size = max_values;
+    if (MultiplyWithOverflow(decoded_values_size,
+                             descr_->type_length() * sizeof(ByteArray),

Review Comment:
   What are you trying to compute here exactly?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3196,6 +3471,45 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+  using Base::pool_;
+
+  int Decode(ByteArray* buffer, int max_values) {
+    return GetInternal(buffer, max_values);
+  }
+  int Decode(FixedLenByteArray* buffer, int max_values) override {
+    int decoded_values_size = max_values;
+    if (MultiplyWithOverflow(decoded_values_size,
+                             descr_->type_length() * sizeof(ByteArray),
+                             &decoded_values_size)) {
+      throw ParquetException("excess expansion in DELTA_LENGTH_BYTE_ARRAY");
+    }
+    ArrowPoolVector<uint8_t> decode_values(decoded_values_size,
+                                           ::arrow::stl::allocator<uint8_t>(pool_));
+    auto decode_buf = reinterpret_cast<ByteArray*>(decode_values.data());
+
+    max_values = GetInternal(decode_buf, max_values);
+    for (int i = 0; i < max_values; i++) {
+      buffer[i].ptr = decode_buf->ptr + i * descr_->type_length();

Review Comment:
   Hmm, how is this supposed to work? `decode_buf` is an array of `ByteArray` structures, not some string data. `decode_buf->ptr` is equal to `decode_buf.ptr[0]`. Also this assumes that all strings have the expected length...



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,293 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void CheckRoundtrip() override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());

Review Comment:
   What is false? Please add the parameter name.
   ```suggestion
           MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, /*xxx=*/ false, descr_.get());
   ```



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,243 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw ParquetException("Put not implemented for " + this->descr_->ToString());
+}
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i++) {
+    // Convert to ByteArray, so we can pass to the suffix_encoder_.
+    const ByteArray value = src[i];
+    if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+      throw ParquetException("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value.ptr),
+                            static_cast<uint32_t>(value.len)};
+    uint32_t j = 0;
+    const uint32_t common_length =
+        std::min(value.len, static_cast<uint32_t>(last_value_view.length()));
+    while (j < common_length) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
+    last_value_view = view;
+    prefix_lengths[i] = j;
+    const auto suffix_length = static_cast<uint32_t>(value.len - j);
+
+    if (suffix_length == 0) {
+      suffix_encoder_.Put(&kEmpty, 1);
+      continue;
+    }
+    const uint8_t* suffix_ptr = value.ptr + j;
+    // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+    const ByteArray suffix(suffix_length, suffix_ptr);
+    suffix_encoder_.Put(&suffix, 1);
+  }
+  prefix_length_encoder_.Put(prefix_lengths.data(), num_values);
+  last_value_ = last_value_view;
+}
+
+template <>
+void DeltaByteArrayEncoder<FLBAType>::Put(const FLBA* src, int num_values) {

Review Comment:
   The duplication here should be quite easily avoided...



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3196,6 +3471,45 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+  using Base::pool_;
+
+  int Decode(ByteArray* buffer, int max_values) {

Review Comment:
   It seems unexpected to decode to `ByteArray*` if the Parquet type is `FLBAType`. Is this necessary?



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,293 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void CheckRoundtrip() override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType, FLBAType> TestDeltaByteArrayEncodingTypes;

Review Comment:
   Style nit :-)
   ```suggestion
   using TestDeltaByteArrayEncodingTypes = ::testing::Types<ByteArrayType, FLBAType>;
   ```



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,293 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void CheckRoundtrip() override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType, FLBAType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 2));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_prob*/ 0));
+
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 200));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.1));
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPut) {
+  const int64_t size = 50;
+  const int32_t min_length = 0;
+  const int32_t max_length = 10;
+  const int32_t num_unique = 10;
+  const double null_probability = 0.25;
+  auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());

Review Comment:
   Hmm, I'm not sure what this is and `upcast_result` isn't used below anyway.



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,293 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void CheckRoundtrip() override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType, FLBAType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 2));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_prob*/ 0));
+
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 200));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.1));
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPut) {
+  const int64_t size = 50;
+  const int32_t min_length = 0;
+  const int32_t max_length = 10;
+  const int32_t num_unique = 10;
+  const double null_probability = 0.25;
+  auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  ::arrow::random::RandomArrayGenerator rag(42);
+  auto values = rag.String(0, min_length, max_length, null_probability);
+  CheckSeed(values);
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    rag = ::arrow::random::RandomArrayGenerator(seed);
+
+    values = rag.String(size, min_length, max_length, null_probability);
+    CheckSeed(values);
+
+    values =
+        rag.BinaryWithRepeats(size, num_unique, min_length, max_length, null_probability);
+    CheckSeed(values);
+  }
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPutFixedLength) {
+  const int64_t size = 50;
+  const double null_probability = 0.25;
+  ::arrow::random::RandomArrayGenerator rag(0);
+  auto encoder = MakeTypedEncoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<FLBAType>::Accumulator acc(values->type());
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    for (auto length : {0, 10, 100, 1000}) {
+      rag = ::arrow::random::RandomArrayGenerator(seed);
+      auto values = rag.FixedSizeBinary(size, length, null_probability);
+      CheckSeed(values);
+    }
+  }
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
+  auto CheckEncode = [](std::shared_ptr<::arrow::Array> values,
+                        std::shared_ptr<::arrow::Array> prefix_lengths,
+                        std::shared_ptr<::arrow::Array> suffix_lengths,
+                        std::string_view value) {
+    auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    auto prefix_lengths_encoder =
+        MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
+    ASSERT_NO_THROW(prefix_lengths_encoder->Put(*prefix_lengths));
+    auto prefix_lengths_buf = prefix_lengths_encoder->FlushValues();
+
+    auto encoded_prefix_lengths_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
+
+    auto suffix_lengths_encoder =
+        MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
+    ASSERT_NO_THROW(suffix_lengths_encoder->Put(*suffix_lengths));
+    auto suffix_lengths_buf = suffix_lengths_encoder->FlushValues();
+    auto encoded_values_buf =
+        SliceBuffer(buf, prefix_lengths_buf->size() + suffix_lengths_buf->size());
+
+    auto encoded_prefix_length_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
+    EXPECT_TRUE(prefix_lengths_buf->Equals(*encoded_prefix_length_buf));
+    auto encoded_suffix_length_buf =
+        SliceBuffer(buf, prefix_lengths_buf->size(), suffix_lengths_buf->size());
+    EXPECT_TRUE(suffix_lengths_buf->Equals(*encoded_suffix_length_buf));
+    EXPECT_EQ(value, encoded_values_buf->ToString());
+  };
+
+  auto arrayToI32 = [](const std::shared_ptr<::arrow::Array>& lengths) {
+    std::vector<int32_t> arrays;
+    auto data_ptr = checked_cast<::arrow::Int32Array*>(lengths.get());
+    for (int i = 0; i < lengths->length(); ++i) {
+      arrays.push_back(data_ptr->GetView(i));
+    }
+    return arrays;
+  };
+
+  auto CheckDecode = [](std::shared_ptr<Buffer> buf,
+                        std::shared_ptr<::arrow::Array> values) {
+    int num_values = static_cast<int>(values->length());
+    auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(num_values, result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *upcast_result);
+  };
+
+  auto CheckEncodeDecode = [&](std::string_view values,
+                               std::shared_ptr<::arrow::Array> prefix_lengths,
+                               std::shared_ptr<::arrow::Array> suffix_lengths,
+                               std::string_view suffix_data) {
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::utf8(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::large_utf8(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::binary(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::large_binary(), values), prefix_lengths,
+                suffix_lengths, suffix_data);

Review Comment:
   I think `CheckEncode` can be simplified if you make it take the `encoded` buffer computed below and just compare it with the actual encoded results.



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,293 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void CheckRoundtrip() override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType, FLBAType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 2));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_prob*/ 0));
+
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 200));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.1));
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPut) {
+  const int64_t size = 50;
+  const int32_t min_length = 0;
+  const int32_t max_length = 10;
+  const int32_t num_unique = 10;
+  const double null_probability = 0.25;
+  auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  ::arrow::random::RandomArrayGenerator rag(42);
+  auto values = rag.String(0, min_length, max_length, null_probability);
+  CheckSeed(values);
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    rag = ::arrow::random::RandomArrayGenerator(seed);
+
+    values = rag.String(size, min_length, max_length, null_probability);
+    CheckSeed(values);
+
+    values =
+        rag.BinaryWithRepeats(size, num_unique, min_length, max_length, null_probability);
+    CheckSeed(values);
+  }
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPutFixedLength) {
+  const int64_t size = 50;

Review Comment:
   You probably want to reduce code duplication between this and the previous test.



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,293 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void CheckRoundtrip() override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType, FLBAType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 2));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_prob*/ 0));
+
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 200));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.1));
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPut) {
+  const int64_t size = 50;
+  const int32_t min_length = 0;
+  const int32_t max_length = 10;
+  const int32_t num_unique = 10;
+  const double null_probability = 0.25;
+  auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  ::arrow::random::RandomArrayGenerator rag(42);
+  auto values = rag.String(0, min_length, max_length, null_probability);
+  CheckSeed(values);
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    rag = ::arrow::random::RandomArrayGenerator(seed);
+
+    values = rag.String(size, min_length, max_length, null_probability);
+    CheckSeed(values);
+
+    values =
+        rag.BinaryWithRepeats(size, num_unique, min_length, max_length, null_probability);
+    CheckSeed(values);
+  }
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPutFixedLength) {
+  const int64_t size = 50;
+  const double null_probability = 0.25;
+  ::arrow::random::RandomArrayGenerator rag(0);
+  auto encoder = MakeTypedEncoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<FLBAType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<FLBAType>::Accumulator acc(values->type());
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+    for (auto length : {0, 10, 100, 1000}) {
+      rag = ::arrow::random::RandomArrayGenerator(seed);
+      auto values = rag.FixedSizeBinary(size, length, null_probability);
+      CheckSeed(values);
+    }
+  }
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
+  auto CheckEncode = [](std::shared_ptr<::arrow::Array> values,
+                        std::shared_ptr<::arrow::Array> prefix_lengths,
+                        std::shared_ptr<::arrow::Array> suffix_lengths,
+                        std::string_view value) {
+    auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    auto prefix_lengths_encoder =
+        MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
+    ASSERT_NO_THROW(prefix_lengths_encoder->Put(*prefix_lengths));
+    auto prefix_lengths_buf = prefix_lengths_encoder->FlushValues();
+
+    auto encoded_prefix_lengths_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
+
+    auto suffix_lengths_encoder =
+        MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
+    ASSERT_NO_THROW(suffix_lengths_encoder->Put(*suffix_lengths));
+    auto suffix_lengths_buf = suffix_lengths_encoder->FlushValues();
+    auto encoded_values_buf =
+        SliceBuffer(buf, prefix_lengths_buf->size() + suffix_lengths_buf->size());
+
+    auto encoded_prefix_length_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
+    EXPECT_TRUE(prefix_lengths_buf->Equals(*encoded_prefix_length_buf));
+    auto encoded_suffix_length_buf =
+        SliceBuffer(buf, prefix_lengths_buf->size(), suffix_lengths_buf->size());
+    EXPECT_TRUE(suffix_lengths_buf->Equals(*encoded_suffix_length_buf));
+    EXPECT_EQ(value, encoded_values_buf->ToString());
+  };
+
+  auto arrayToI32 = [](const std::shared_ptr<::arrow::Array>& lengths) {
+    std::vector<int32_t> arrays;
+    auto data_ptr = checked_cast<::arrow::Int32Array*>(lengths.get());
+    for (int i = 0; i < lengths->length(); ++i) {
+      arrays.push_back(data_ptr->GetView(i));
+    }
+    return arrays;
+  };
+
+  auto CheckDecode = [](std::shared_ptr<Buffer> buf,
+                        std::shared_ptr<::arrow::Array> values) {
+    int num_values = static_cast<int>(values->length());
+    auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(num_values, result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *upcast_result);
+  };
+
+  auto CheckEncodeDecode = [&](std::string_view values,
+                               std::shared_ptr<::arrow::Array> prefix_lengths,
+                               std::shared_ptr<::arrow::Array> suffix_lengths,
+                               std::string_view suffix_data) {
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::utf8(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::large_utf8(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::binary(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+    CheckEncode(::arrow::ArrayFromJSON(::arrow::large_binary(), values), prefix_lengths,
+                suffix_lengths, suffix_data);
+
+    auto encoded = ::arrow::ConcatenateBuffers({DeltaEncode(arrayToI32(prefix_lengths)),
+                                                DeltaEncode(arrayToI32(suffix_lengths)),
+                                                std::make_shared<Buffer>(suffix_data)})
+                       .ValueOrDie();
+
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::utf8(), values));
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_utf8(), values));
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::binary(), values));
+    CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
+  };
+
+  {
+    auto values = R"(["axis", "axle", "babble", "babyhood"])";
+    auto prefix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([0, 2, 0, 3])");
+    auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([4, 2, 6, 5])");
+
+    constexpr std::string_view suffix_data = "axislebabbleyhood";
+    CheckEncodeDecode(values, prefix_lengths, suffix_lengths, suffix_data);
+  }
+
+  {
+    auto values = R"(["axis", "axis", "axis", "axis"])";
+    auto prefix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([0, 4, 4, 4])");
+    auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([4, 0, 0, 0])");
+
+    constexpr std::string_view suffix_data = "axis";
+    CheckEncodeDecode(values, prefix_lengths, suffix_lengths, suffix_data);
+  }
+
+  {
+    auto values = R"(["axisba", "axis", "axis", "axis"])";
+    auto prefix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([0, 4, 4, 4])");
+    auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([6, 0, 0, 0])");
+
+    constexpr std::string_view suffix_data = "axisba";
+    CheckEncodeDecode(values, prefix_lengths, suffix_lengths, suffix_data);
+  }
+
+  {
+    auto values = R"(["baaxis", "axis", "axis", "axis"])";
+    auto prefix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([0, 0, 4, 4])");
+    auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([6, 4, 0, 0])");
+
+    constexpr std::string_view suffix_data = "baaxisaxis";
+    CheckEncodeDecode(values, prefix_lengths, suffix_lengths, suffix_data);
+  }
+
+  {
+    auto values = R"(["καλημέρα", "καμηλιέρη", "καμηλιέρη", "καλημέρα"])";
+    auto prefix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([0, 5, 18, 5])");
+    auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([16, 13, 0, 11])");
+    const std::string suffix_data = "καλημέρα\xbcηλιέρη\xbbημέρα";

Review Comment:
   Interesting. The `\x` is because a utf-8 character is split between prefix and suffix, right?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1198693475


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3200,6 +3461,42 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+  using Base::pool_;
+
+  int Decode(FixedLenByteArray* buffer, int max_values) override {

Review Comment:
   That make sense. I included this with a small adjustment:
   ```cpp
     int Decode(FixedLenByteArray* buffer, int max_values) override {
       // GetInternal currently only support ByteArray.
       std::vector<ByteArray> decode_byte_array(max_values);
       const int decoded_values_size = GetInternal(decode_byte_array.data(), max_values);
       const uint32_t type_length = descr_->type_length();
   
       for (int i = 0; i < decoded_values_size; i++) {
         if (ARROW_PREDICT_FALSE(decode_byte_array[i].len != type_length)) {
           throw ParquetException("Fixed length byte array length mismatch");
         }
         buffer[i].ptr = decode_byte_array.data()->ptr + i * type_length;
       }
       return decoded_values_size;
     }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1198695229


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,229 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        auto view = visitor[i + j];
+        len = visitor.len(i + j);
+
+        uint32_t k = 0;
+        const uint32_t common_length =
+            std::min(len, static_cast<uint32_t>(last_value_view.length()));
+        while (k < common_length) {
+          if (last_value_view[k] != view[k]) {
+            break;
+          }
+          k++;
+        }
+
+        last_value_view = view;
+        prefix_lengths[j] = k;
+        const auto suffix_length = len - k;
+        const uint8_t* suffix_ptr = src[i + j].ptr + k;
+
+        // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+        const ByteArray suffix(suffix_length, suffix_ptr);
+        suffixes[j] = suffix;
+      }
+      suffix_encoder_.Put(suffixes.data(), batch_size);
+      prefix_length_encoder_.Put(prefix_lengths.data(), batch_size);
+    }
+    last_value_ = last_value_view;
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+struct ByteArrayVisitor {
+  const ByteArray* src;
+  const uint32_t length;
+
+  std::string_view operator[](int i) const {
+    if (ARROW_PREDICT_FALSE(src[i].len >= kMaxByteArraySize)) {
+      throw ParquetException("Parquet cannot store strings with size 2GB or more");
+    }
+    return std::string_view{src[i]};
+  }
+
+  const uint32_t len(int i) const { return src[i].len; }
+};
+
+struct FLBAVisitor {
+  const FLBA* src;
+  const uint32_t length;
+
+  std::string_view operator[](int i) const {
+    return std::string_view{reinterpret_cast<const char*>(src[i].ptr)};

Review Comment:
   Indeed! Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1198835972


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,229 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        auto view = visitor[i + j];
+        len = visitor.len(i + j);
+
+        uint32_t k = 0;
+        const uint32_t common_length =
+            std::min(len, static_cast<uint32_t>(last_value_view.length()));
+        while (k < common_length) {
+          if (last_value_view[k] != view[k]) {
+            break;
+          }
+          k++;
+        }
+
+        last_value_view = view;
+        prefix_lengths[j] = k;
+        const auto suffix_length = len - k;
+        const uint8_t* suffix_ptr = src[i + j].ptr + k;
+
+        // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+        const ByteArray suffix(suffix_length, suffix_ptr);
+        suffixes[j] = suffix;
+      }
+      suffix_encoder_.Put(suffixes.data(), batch_size);
+      prefix_length_encoder_.Put(prefix_lengths.data(), batch_size);
+    }
+    last_value_ = last_value_view;
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+struct ByteArrayVisitor {
+  const ByteArray* src;
+  const uint32_t length;
+
+  std::string_view operator[](int i) const {
+    if (ARROW_PREDICT_FALSE(src[i].len >= kMaxByteArraySize)) {
+      throw ParquetException("Parquet cannot store strings with size 2GB or more");
+    }
+    return std::string_view{src[i]};
+  }
+
+  uint32_t len(int i) const { return src[i].len; }
+};
+
+struct FLBAVisitor {
+  const FLBA* src;
+  const uint32_t length;

Review Comment:
   Good point! Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1295889868


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1908,4 +1909,249 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestDeltaLengthByteArrayEncoding<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+  static constexpr double prefixed_probability = 0.5;
+
+  void InitData(int nvalues, int repeats) {

Review Comment:
   I just noticed the base `InitData` isn't marked `virtual`, so it's not obvious if this derived method will be called instead. Can you mark the base `InitData` `virtual` and make this one `override`?
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1299985895


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,43 +1240,93 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
-  explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
-    this->out = out;
-    this->builder = out->builder.get();
-    this->chunk_space_remaining =
-        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+template <typename DType, typename Enable = void>
+struct ArrowBinaryHelper;
+
+template <typename DType>
+struct ArrowBinaryHelper<DType,
+                         std::enable_if_t<std::is_same_v<DType, ByteArrayType>, void>> {
+  explicit ArrowBinaryHelper(typename EncodingTraits<DType>::Accumulator* acc) {
+    builder = acc->builder.get();
+    chunks = &acc->chunks;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 builder->value_data_length(),
+                                                 &chunk_space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<DType>");
+    }
   }
 
   Status PushChunk() {
     std::shared_ptr<::arrow::Array> result;
     RETURN_NOT_OK(builder->Finish(&result));
-    out->chunks.push_back(result);
+    chunks->push_back(result);
     chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
     return Status::OK();
   }
 
   bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
 
   void UnsafeAppend(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
     builder->UnsafeAppend(data, length);
   }
 
   void UnsafeAppendNull() { builder->UnsafeAppendNull(); }
 
   Status Append(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
     return builder->Append(data, length);
   }
 
   Status AppendNull() { return builder->AppendNull(); }
 
-  typename EncodingTraits<ByteArrayType>::Accumulator* out;
-  ::arrow::BinaryBuilder* builder;
+  typename EncodingTraits<DType>::BuilderType* builder;
+  std::vector<std::shared_ptr<::arrow::Array>>* chunks;
   int64_t chunk_space_remaining;
 };
 
+template <typename DType>
+struct ArrowBinaryHelper<DType, std::enable_if_t<std::is_same_v<DType, FLBAType>, void>> {
+  explicit ArrowBinaryHelper(typename EncodingTraits<DType>::Accumulator* acc) {
+    builder = acc;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 builder->value_data_length(),
+                                                 &space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<DType>");
+    }
+  }
+
+  Status PushChunk() {
+    std::shared_ptr<::arrow::Array> result;
+    RETURN_NOT_OK(builder->Finish(&result));

Review Comment:
   Another option would be to refactor `ArrowBinaryHelper` to something like `ArrowBinaryHelper<DType>(EncodingTraits<DType>::BuilderType, std::vector<std::shared_ptr<::arrow::Array>>*)`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1300152071


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,40 +1240,54 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
+template <typename DType>
 struct ArrowBinaryHelper {
-  explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
-    this->out = out;
-    this->builder = out->builder.get();
-    this->chunk_space_remaining =
-        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+  explicit ArrowBinaryHelper(typename EncodingTraits<DType>::Accumulator* acc) {
+    if constexpr (std::is_same_v<DType, ByteArrayType>) {
+      builder = acc->builder.get();
+      chunks = &acc->chunks;
+    } else {
+      builder = acc;
+    }
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 builder->value_data_length(),
+                                                 &chunk_space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<DType>");
+    }
   }
 
   Status PushChunk() {
     std::shared_ptr<::arrow::Array> result;
     RETURN_NOT_OK(builder->Finish(&result));
-    out->chunks.push_back(result);
+    chunks->push_back(result);
     chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
     return Status::OK();
   }
 
   bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
 
   void UnsafeAppend(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
     builder->UnsafeAppend(data, length);
   }
 
   void UnsafeAppendNull() { builder->UnsafeAppendNull(); }
 
   Status Append(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
-    return builder->Append(data, length);
+    if constexpr (std::is_same_v<DType, FLBAType>) {
+      return builder->Append(data);
+    } else {
+      return builder->Append(data, length);
+    }
   }
 
   Status AppendNull() { return builder->AppendNull(); }
 
-  typename EncodingTraits<ByteArrayType>::Accumulator* out;
-  ::arrow::BinaryBuilder* builder;
+  typename EncodingTraits<DType>::BuilderType* builder;
+  std::vector<std::shared_ptr<::arrow::Array>>* chunks;

Review Comment:
   Well, it's nice that you have a `chunks` member variable... except that `chunks` won't be retrieved when the input is FLBA, because the user is only passing a `BinaryBuilder`. So some data will be lost if it exceeds 2GB.
   
   I'll try to come back with a better approach and push it here, in the interest of saving us more back and forths :-)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1300229433


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,40 +1240,54 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
+template <typename DType>
 struct ArrowBinaryHelper {
-  explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
-    this->out = out;
-    this->builder = out->builder.get();
-    this->chunk_space_remaining =
-        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+  explicit ArrowBinaryHelper(typename EncodingTraits<DType>::Accumulator* acc) {
+    if constexpr (std::is_same_v<DType, ByteArrayType>) {
+      builder = acc->builder.get();
+      chunks = &acc->chunks;
+    } else {
+      builder = acc;
+    }
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 builder->value_data_length(),
+                                                 &chunk_space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<DType>");
+    }
   }
 
   Status PushChunk() {
     std::shared_ptr<::arrow::Array> result;
     RETURN_NOT_OK(builder->Finish(&result));
-    out->chunks.push_back(result);
+    chunks->push_back(result);
     chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
     return Status::OK();
   }
 
   bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
 
   void UnsafeAppend(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
     builder->UnsafeAppend(data, length);
   }
 
   void UnsafeAppendNull() { builder->UnsafeAppendNull(); }
 
   Status Append(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
-    return builder->Append(data, length);
+    if constexpr (std::is_same_v<DType, FLBAType>) {
+      return builder->Append(data);
+    } else {
+      return builder->Append(data, length);
+    }
   }
 
   Status AppendNull() { return builder->AppendNull(); }
 
-  typename EncodingTraits<ByteArrayType>::Accumulator* out;
-  ::arrow::BinaryBuilder* builder;
+  typename EncodingTraits<DType>::BuilderType* builder;
+  std::vector<std::shared_ptr<::arrow::Array>>* chunks;

Review Comment:
   No.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] conbench-apache-arrow[bot] commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "conbench-apache-arrow[bot] (via GitHub)" <gi...@apache.org>.
conbench-apache-arrow[bot] commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1687397488

   After merging your PR, Conbench analyzed the 6 benchmarking runs that have been run so far on merge-commit 94bd0d27326e8f3bfc0f612fcf604617402888ec.
   
   There were no benchmark performance regressions. 🎉
   
   The [full Conbench report](https://github.com/apache/arrow/runs/16091343646) has more details. It also includes information about possible false positives for unstable benchmarks that are known to sometimes produce them.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1687881814

   Feel free to open a PR :-)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1285951190


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -874,7 +874,7 @@ std::shared_ptr<::arrow::Array> EncodingAdHocTyped<FLBAType>::GetValues(int seed
 }
 
 using EncodingAdHocTypedCases =
-    ::testing::Types<BooleanType, Int32Type, Int64Type, FloatType, DoubleType, FLBAType>;
+    ::testing::Types<BooleanType, Int32Type, Int64Type, FloatType, DoubleType>;

Review Comment:
   Yes, the change to `EncodingTraits<FLBAType>` breaks `EncodingAdHocTyped` test. Specifically we go from:
   ```cpp
   struct EncodingTraits<FLBAType> {
     using Accumulator = ::arrow::FixedSizeBinaryBuilder;
     [...]
   ```
   to:
   ```cpp
   struct EncodingTraits<FLBAType> {
     struct Accumulator {
       std::unique_ptr<::arrow::FixedSizeBinaryBuilder> builder;
       std::vector<std::shared_ptr<::arrow::Array>> chunks;
     [...]
   ```
   If we consider `EncodingTraits` external API we should probably avoid this change.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1286004321


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3083,242 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          buffer_, ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));

Review Comment:
   How about changing to `ResizableBuffer` and:
   ```cpp
         if (buffer_ == nullptr) {
           PARQUET_ASSIGN_OR_THROW(
               buffer_, ::arrow::AllocateResizableBuffer(num_values * sizeof(T), this->memory_pool()));
         } else {
           PARQUET_THROW_NOT_OK(buffer_->Resize(num_values * sizeof(T), false));
         }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1297152102


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,230 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        auto view = visitor[i + j];
+        len = visitor.len(i + j);
+
+        uint32_t k = 0;
+        const uint32_t common_length =
+            std::min(len, static_cast<uint32_t>(last_value_view.length()));
+        while (k < common_length) {
+          if (last_value_view[k] != view[k]) {
+            break;
+          }
+          k++;
+        }
+
+        last_value_view = view;
+        prefix_lengths[j] = k;
+        const uint32_t suffix_length = len - k;
+        const uint8_t* suffix_ptr = src[i + j].ptr + k;
+
+        // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+        const ByteArray suffix(suffix_length, suffix_ptr);
+        suffixes[j] = suffix;
+      }
+      suffix_encoder_.Put(suffixes.data(), batch_size);
+      prefix_length_encoder_.Put(prefix_lengths.data(), batch_size);
+    }
+    last_value_ = last_value_view;
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.length());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t len = src.len;
+          const uint32_t common_length = std::min(previous_len, len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+struct ByteArrayVisitor {
+  const ByteArray* src;
+  const uint32_t type_length;

Review Comment:
   Oh, indeed. Changed to your suggestion.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1264212141


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -874,7 +874,7 @@ std::shared_ptr<::arrow::Array> EncodingAdHocTyped<FLBAType>::GetValues(int seed
 }
 
 using EncodingAdHocTypedCases =
-    ::testing::Types<BooleanType, Int32Type, Int64Type, FloatType, DoubleType, FLBAType>;
+    ::testing::Types<BooleanType, Int32Type, Int64Type, FloatType, DoubleType>;

Review Comment:
   @wgtmac [proposed it](https://github.com/apache/arrow/pull/14341#discussion_r1256021740) to simplify `ArrowBinaryHelper` and there was no discussion yet. Would this be considered an external API change?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3058,235 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+  static constexpr std::string_view kEmpty = "";
+
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(descr, pool),
+        last_value_(""),
+        empty_(static_cast<uint32_t>(kEmpty.size()),
+               reinterpret_cast<const uint8_t*>(kEmpty.data())) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        const int idx = i + j;
+        auto view = visitor[idx];
+        len = visitor.len(idx);
+
+        uint32_t k = 0;
+        const uint32_t common_length =

Review Comment:
   Changed.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3058,235 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+  static constexpr std::string_view kEmpty = "";
+
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(descr, pool),
+        last_value_(""),
+        empty_(static_cast<uint32_t>(kEmpty.size()),
+               reinterpret_cast<const uint8_t*>(kEmpty.data())) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        const int idx = i + j;
+        auto view = visitor[idx];
+        len = visitor.len(idx);

Review Comment:
   `const uint32_t len = view.length()` is an option.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3058,235 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+  static constexpr std::string_view kEmpty = "";
+
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(descr, pool),
+        last_value_(""),
+        empty_(static_cast<uint32_t>(kEmpty.size()),
+               reinterpret_cast<const uint8_t*>(kEmpty.data())) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();

Review Comment:
   Changed.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,43 +1240,62 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
-  explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
-    this->out = out;
-    this->builder = out->builder.get();
-    this->chunk_space_remaining =
-        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+template <typename DType, typename Enable = void>
+struct ArrowBinaryHelper;
+
+template <typename DType>
+struct ArrowBinaryHelper<DType, std::enable_if_t<std::is_same_v<DType, ByteArrayType> ||
+                                                     std::is_same_v<DType, FLBAType>,
+                                                 void>> {
+  explicit ArrowBinaryHelper(typename EncodingTraits<DType>::Accumulator* acc) {
+    builder = acc->builder.get();
+    chunks = acc->chunks;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 builder->value_data_length(),
+                                                 &chunk_space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<DType>");
+    }
   }
 
   Status PushChunk() {
     std::shared_ptr<::arrow::Array> result;
     RETURN_NOT_OK(builder->Finish(&result));
-    out->chunks.push_back(result);
+    chunks.push_back(std::move(result));
     chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
     return Status::OK();
   }
 
   bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
 
   void UnsafeAppend(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
     builder->UnsafeAppend(data, length);
   }
 
   void UnsafeAppendNull() { builder->UnsafeAppendNull(); }
 
-  Status Append(const uint8_t* data, int32_t length) {
-    chunk_space_remaining -= length;
-    return builder->Append(data, length);
-  }
+  virtual Status Append(const uint8_t* data, int32_t length);
 
   Status AppendNull() { return builder->AppendNull(); }
 
-  typename EncodingTraits<ByteArrayType>::Accumulator* out;
-  ::arrow::BinaryBuilder* builder;
+  typename EncodingTraits<DType>::BuilderType* builder;
+  std::vector<std::shared_ptr<::arrow::Array>> chunks;

Review Comment:
   I prefer the second option. Changed.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3058,235 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+  static constexpr std::string_view kEmpty = "";
+
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),

Review Comment:
   I'm not sure what to comment here. `"// Prefix lengths are encoded using DeltaBitPackEncoder that can be left uninitialized."`?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3058,235 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+  static constexpr std::string_view kEmpty = "";
+
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(descr, pool),
+        last_value_(""),
+        empty_(static_cast<uint32_t>(kEmpty.size()),
+               reinterpret_cast<const uint8_t*>(kEmpty.data())) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        const int idx = i + j;
+        auto view = visitor[idx];
+        len = visitor.len(idx);
+
+        uint32_t k = 0;
+        const uint32_t common_length =
+            std::min(len, static_cast<uint32_t>(last_value_view.length()));
+        while (k < common_length) {
+          if (last_value_view[k] != view[k]) {
+            break;
+          }
+          k++;
+        }
+
+        last_value_view = view;
+        prefix_lengths[j] = k;
+        const uint32_t suffix_length = len - k;
+        const uint8_t* suffix_ptr = src[idx].ptr + k;
+
+        // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+        const ByteArray suffix(suffix_length, suffix_ptr);
+        suffixes[j] = suffix;
+      }
+      suffix_encoder_.Put(suffixes.data(), batch_size);
+      prefix_length_encoder_.Put(prefix_lengths.data(), batch_size);
+    }
+    last_value_ = last_value_view;
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.length());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t len = src.len;
+          const uint32_t common_length = std::min(previous_len, len);

Review Comment:
   Changed `common_length` to `maximum_common_prefix_length`. Shall we change `len` to `length`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1271172970


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),

Review Comment:
   Sorry I seem to have missed it. Added.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1635733954

   Rebased to resolve merge conflict.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1149423665


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,180 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = j;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          const uint8_t* suffix_ptr = src.ptr + j;
+          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+          last_value_view =
+              string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};

Review Comment:
   ```suggestion
             last_value_view = view;
   ```



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3196,6 +3397,30 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) {
+    return GetInternal(buffer, max_values);
+  }
+  int Decode(FixedLenByteArray* buffer, int max_values) override {

Review Comment:
   Yes, I think maybe check-and-cast is ok here, because `DeltaByteArrayFLBADecoder` cannot decode FLBA is really weird.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,180 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = j;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          const uint8_t* suffix_ptr = src.ptr + j;
+          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+          last_value_view =
+              string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+          // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  int i = 0;
+  while (i < num_values) {
+    // Convert to ByteArray so we can pass to the suffix_encoder_.
+    auto value = reinterpret_cast<const ByteArray*>(&src[i]);
+    if (ARROW_PREDICT_FALSE(value->len >= kMaxByteArraySize)) {
+      throw Status::Invalid("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value->ptr), value->len};
+    uint32_t j = 0;
+    while (j < std::min(value->len, static_cast<uint32_t>(last_value_view.length()))) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
+    prefix_lengths[i] = j;
+    const uint8_t* suffix_ptr = value->ptr + j;
+    const uint32_t suffix_length = static_cast<uint32_t>(value->len - j);
+    last_value_view =
+        string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};

Review Comment:
   ```suggestion
       last_value_view = view;
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1143965790


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1268,6 +1268,35 @@ struct ArrowBinaryHelper {
   int64_t chunk_space_remaining;
 };
 
+struct ArrowFLBAHelper {
+  explicit ArrowFLBAHelper(::arrow::FixedSizeBinaryBuilder* builder) {
+    this->builder = builder;
+    this->chunk_space_remaining =
+        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+  }
+
+  Status PushChunk() {
+    std::shared_ptr<::arrow::Array> result;
+    RETURN_NOT_OK(builder->Finish(&result));
+    chunks.push_back(result);
+    chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
+    return Status::OK();
+  }
+
+  bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
+
+  Status Append(const uint8_t* data, int32_t length) {
+    chunk_space_remaining -= length;

Review Comment:
   It's checked [`here`](https://github.com/apache/arrow/pull/14341/files#diff-c40f77b93d252f84bd20a4c15130965bedf39adede736bbfe892a2252c39b4f5R3274) same as `ArrowBinaryHelper`. Would you prefer to move both into `Append` method?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1162511592


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,244 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          auto buffer_,
+          ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());

Review Comment:
   Changed.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,244 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          auto buffer_,
+          ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw ParquetException("Put not implemented for " + this->descr_->ToString());
+}
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i++) {
+    // Convert to ByteArray, so we can pass to the suffix_encoder_.
+    const ByteArray value = src[i];
+    if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+      throw ParquetException("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value.ptr),
+                            static_cast<uint32_t>(value.len)};
+    uint32_t j = 0;
+    const uint32_t common_length =
+        std::min(value.len, static_cast<uint32_t>(last_value_view.length()));
+    while (j < common_length) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
+    last_value_view = view;
+    prefix_lengths[i] = j;
+    const auto suffix_length = static_cast<uint32_t>(value.len - j);
+
+    if (suffix_length == 0) {
+      suffix_encoder_.Put(&kEmpty, 1);
+      continue;
+    }
+    const uint8_t* suffix_ptr = value.ptr + j;
+    // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+    const ByteArray suffix(suffix_length, suffix_ptr);
+    suffix_encoder_.Put(&suffix, 1);
+  }
+  prefix_length_encoder_.Put(prefix_lengths.data(), num_values);
+  last_value_ = last_value_view;
+}
+
+template <>
+void DeltaByteArrayEncoder<FLBAType>::Put(const FLBA* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+  const int32_t len = descr_->type_length();
+
+  if (ARROW_PREDICT_FALSE(len >= static_cast<int32_t>(kMaxByteArraySize))) {
+    throw Status::Invalid("Parquet cannot store strings with size 2GB or more");

Review Comment:
   Oh, I missed that one.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1162246099


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,244 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          auto buffer_,
+          ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw ParquetException("Put not implemented for " + this->descr_->ToString());
+}
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i++) {
+    // Convert to ByteArray, so we can pass to the suffix_encoder_.
+    const ByteArray value = src[i];
+    if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+      throw ParquetException("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value.ptr),
+                            static_cast<uint32_t>(value.len)};
+    uint32_t j = 0;
+    const uint32_t common_length =
+        std::min(value.len, static_cast<uint32_t>(last_value_view.length()));
+    while (j < common_length) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
+    last_value_view = view;
+    prefix_lengths[i] = j;
+    const auto suffix_length = static_cast<uint32_t>(value.len - j);
+
+    if (suffix_length == 0) {
+      suffix_encoder_.Put(&kEmpty, 1);
+      continue;
+    }
+    const uint8_t* suffix_ptr = value.ptr + j;
+    // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+    const ByteArray suffix(suffix_length, suffix_ptr);
+    suffix_encoder_.Put(&suffix, 1);
+  }
+  prefix_length_encoder_.Put(prefix_lengths.data(), num_values);
+  last_value_ = last_value_view;
+}
+
+template <>
+void DeltaByteArrayEncoder<FLBAType>::Put(const FLBA* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+  const int32_t len = descr_->type_length();
+
+  if (ARROW_PREDICT_FALSE(len >= static_cast<int32_t>(kMaxByteArraySize))) {
+    throw Status::Invalid("Parquet cannot store strings with size 2GB or more");

Review Comment:
   Seems here still throws Status...



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,244 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          auto buffer_,
+          ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());

Review Comment:
   Should rename `buffer_` to `buffer`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1181840939


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,25 +1240,35 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
+template <typename DType>
+struct ArrowBinaryHelper;
+
+template <>
+struct ArrowBinaryHelper<ByteArrayType> {
   explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
     this->out = out;
     this->builder = out->builder.get();
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 this->builder->value_data_length(),
+                                                 &this->chunk_space_remaining))) {
+      throw ParquetException("excess expansion in DELTA_LENGTH_BYTE_ARRAY");

Review Comment:
   Does this encoding name correct?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -1275,6 +1287,40 @@ struct ArrowBinaryHelper {
   int64_t chunk_space_remaining;
 };
 
+template <>
+struct ArrowBinaryHelper<FLBAType> {
+  explicit ArrowBinaryHelper(EncodingTraits<FLBAType>::Accumulator* builder) {
+    this->builder = builder;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 this->builder->value_data_length(),
+                                                 &this->chunk_space_remaining))) {
+      throw ParquetException("excess expansion in DELTA_LENGTH_BYTE_ARRAY");

Review Comment:
   ditto



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1297145710


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1908,4 +1909,249 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestDeltaLengthByteArrayEncoding<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+  static constexpr double prefixed_probability = 0.5;
+
+  void InitData(int nvalues, int repeats) {

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] AlenkaF commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "AlenkaF (via GitHub)" <gi...@apache.org>.
AlenkaF commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1687394453

   Great work everybody, congrats!!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1148409422


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3062,226 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+    uint32_t previous_len = byte_width;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = byte_width;
+          } else {
+            uint32_t j = 0;
+            while (j < previous_len) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(byte_width - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_TRUE(view.size() > kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = src.len;
+          } else {
+            uint32_t j = 0;
+            while (j < std::min(previous_len, src.len)) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  string_view last_value_;
+};
+
+template <>
+inline void DeltaByteArrayEncoder<FLBAType>::Put(const FixedLenByteArray* src,
+                                                 int num_values) {
+  if (descr_->type_length() == 0) {
+    return;
+  }
+  for (int i = 0; i < num_values; ++i) {
+    // Write the result to the output stream
+    DCHECK(src[i].ptr != nullptr) << "Value ptr cannot be NULL";
+    PARQUET_THROW_NOT_OK(sink_.Append(src[i].ptr, descr_->type_length()));
+  }
+}
+
+template <>
+void DeltaByteArrayEncoder<FLBAType>::Put(const ::arrow::Array& values) {
+  if (!::arrow::is_fixed_size_binary(values.type_id())) {
+    throw ParquetException("Only FixedSizeBinaryArray and subclasses supported");
+  }
+  PutFixedLenByteArray(checked_cast<const ::arrow::FixedSizeBinaryArray&>(values));
+}
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const ::arrow::Array& values) {
+  AssertBaseBinary(values);
+  if (::arrow::is_binary_like(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::BinaryArray&>(values));
+  } else if (::arrow::is_large_binary_like(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::LargeBinaryArray&>(values));
+  }
+}
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values);
+
+  if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+    last_value_ = string_view{reinterpret_cast<const char*>(src[0].ptr), src[0].len};
+    suffix_encoder_.Put(&src[0], 1);
+    prefix_lengths[0] = 0;
+  }
+
+  for (int32_t i = 1; i < num_values; i++) {

Review Comment:
   Refactored.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1491398768

   I've update a testing here, as an update for https://github.com/apache/arrow/pull/14341#discussion_r1149520662
   
   ```c++
   TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
     auto CheckEncode = [](std::shared_ptr<::arrow::Array> values,
                           std::shared_ptr<::arrow::Array> prefix_lengths,
                           std::shared_ptr<::arrow::Array> suffix_lengths,
                           std::string_view value) {
       auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
       ASSERT_NO_THROW(encoder->Put(*values));
       auto buf = encoder->FlushValues();
   
       auto prefix_lengths_encoder =
           MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
       ASSERT_NO_THROW(prefix_lengths_encoder->Put(*prefix_lengths));
       auto prefix_lengths_buf = prefix_lengths_encoder->FlushValues();
   
       auto encoded_prefix_lengths_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
   
       auto suffix_lengths_encoder =
           MakeTypedEncoder<Int32Type>(Encoding::DELTA_BINARY_PACKED);
       ASSERT_NO_THROW(suffix_lengths_encoder->Put(*suffix_lengths));
       auto suffix_lengths_buf = suffix_lengths_encoder->FlushValues();
       auto encoded_values_buf =
           SliceBuffer(buf, prefix_lengths_buf->size() + suffix_lengths_buf->size());
   
       auto encoded_prefix_length_buf = SliceBuffer(buf, 0, prefix_lengths_buf->size());
       EXPECT_TRUE(prefix_lengths_buf->Equals(*encoded_prefix_length_buf));
       auto encoded_suffix_length_buf =
           SliceBuffer(buf, prefix_lengths_buf->size(), suffix_lengths_buf->size());
       EXPECT_TRUE(suffix_lengths_buf->Equals(*encoded_suffix_length_buf));
       EXPECT_EQ(value, encoded_values_buf->ToString());
     };
   
     auto arrayToI32 = [](const std::shared_ptr<::arrow::Array>& lengths) {
       std::vector<int32_t> arrays;
       auto data_ptr = checked_cast<::arrow::Int32Array*>(lengths.get());
       for (int i = 0; i < lengths->length(); ++i) {
         arrays.push_back(data_ptr->GetView(i));
       }
       return arrays;
     };
   
     auto CheckDecode = [](std::shared_ptr<Buffer> buf,
                           std::shared_ptr<::arrow::Array> values) {
       int num_values = static_cast<int>(values->length());
       auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
       decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
   
       typename EncodingTraits<ByteArrayType>::Accumulator acc;
       if (::arrow::is_string(values->type()->id())) {
         acc.builder = std::make_unique<::arrow::StringBuilder>();
       } else {
         acc.builder = std::make_unique<::arrow::BinaryBuilder>();
       }
   
       ASSERT_EQ(num_values,
                 decoder->DecodeArrow(static_cast<int>(values->length()),
                                      static_cast<int>(values->null_count()),
                                      values->null_bitmap_data(), values->offset(), &acc));
   
       std::shared_ptr<::arrow::Array> result;
       ASSERT_OK(acc.builder->Finish(&result));
       ASSERT_EQ(num_values, result->length());
       ASSERT_OK(result->ValidateFull());
   
       auto upcast_result = CastBinaryTypesHelper(result, values->type());
       ::arrow::AssertArraysEqual(*values, *upcast_result);
     };
   
     auto checkEncodeDecode = [&](std::string_view values,
                                  std::shared_ptr<::arrow::Array> prefix_lengths,
                                  std::shared_ptr<::arrow::Array> suffix_lengths,
                                  std::string_view suffix_data) {
       CheckEncode(::arrow::ArrayFromJSON(::arrow::utf8(), values), prefix_lengths,
                   suffix_lengths, suffix_data);
       CheckEncode(::arrow::ArrayFromJSON(::arrow::large_utf8(), values), prefix_lengths,
                   suffix_lengths, suffix_data);
       CheckEncode(::arrow::ArrayFromJSON(::arrow::binary(), values), prefix_lengths,
                   suffix_lengths, suffix_data);
       CheckEncode(::arrow::ArrayFromJSON(::arrow::large_binary(), values), prefix_lengths,
                   suffix_lengths, suffix_data);
   
       auto encoded = ::arrow::ConcatenateBuffers({DeltaEncode(arrayToI32(prefix_lengths)),
                                                   DeltaEncode(arrayToI32(suffix_lengths)),
                                                   std::make_shared<Buffer>(suffix_data)})
                          .ValueOrDie();
   
       CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::utf8(), values));
       CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_utf8(), values));
       CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::binary(), values));
       CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
     };
   
     {
       auto values = R"(["axis", "axle", "babble", "babyhood"])";
       auto prefix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([0, 2, 0, 3])");
       auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([4, 2, 6, 5])");
   
       constexpr std::string_view suffix_data = "axislebabbleyhood";
       checkEncodeDecode(values, prefix_lengths, suffix_lengths, suffix_data);
     }
   
     {
       auto values = R"(["axis", "axis", "axis", "axis"])";
       auto prefix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([0, 4, 4, 4])");
       auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([4, 0, 0, 0])");
   
       constexpr std::string_view suffix_data = "axis";
       checkEncodeDecode(values, prefix_lengths, suffix_lengths, suffix_data);
     }
   
     {
       auto values = R"(["axisba", "axis", "axis", "axis"])";
       auto prefix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([0, 4, 4, 4])");
       auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([6, 0, 0, 0])");
   
       constexpr std::string_view suffix_data = "axisba";
       checkEncodeDecode(values, prefix_lengths, suffix_lengths, suffix_data);
     }
     {
       auto values = R"(["baaxis", "axis", "axis", "axis"])";
       auto prefix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([0, 0, 4, 4])");
       auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([6, 4, 0, 0])");
   
       constexpr std::string_view suffix_data = "baaxisaxis";
       checkEncodeDecode(values, prefix_lengths, suffix_lengths, suffix_data);
     }
   }
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1195461509


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1966,7 +1966,7 @@ class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
 
   void CheckRoundtrip() override {
     auto encoder =
-        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, /*xxx=*/ false, descr_.get());

Review Comment:
   (Just write name of field, like `use_dictionary`)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1205509102


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3200,6 +3461,39 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+  using Base::pool_;
+
+  int Decode(FixedLenByteArray* buffer, int max_values) override {
+    // GetInternal currently only support ByteArray.
+    std::vector<ByteArray> decode_byte_array(max_values);

Review Comment:
   A pity that this does a potentially large temporary allocation, but that's probably the easiest way given how decoding is currently structured.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1261894605


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -874,7 +874,7 @@ std::shared_ptr<::arrow::Array> EncodingAdHocTyped<FLBAType>::GetValues(int seed
 }
 
 using EncodingAdHocTypedCases =
-    ::testing::Types<BooleanType, Int32Type, Int64Type, FloatType, DoubleType, FLBAType>;
+    ::testing::Types<BooleanType, Int32Type, Int64Type, FloatType, DoubleType>;

Review Comment:
   `FLBAType` was removed due to changes to `EncodingTraits<FLBAType>` above. Perhaps we could do something like:
   ```
     if constexpr (IsFixedSizeBinary::value) {
       using BuilderType = typename EncodingTraits<ParquetType>::BuilderType;
     } else {
       using BuilderType = typename EncodingTraits<ParquetType>::Accumulator;
     }
   ```
   to get around this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1198454687


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,12 +3079,243 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))),
+        prefix_lengths_(kBatchSize_, ::arrow::stl::allocator<int32_t>(pool_)) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+  ArrowPoolVector<int32_t> prefix_lengths_;
+  static constexpr int kBatchSize_ = 256;
+};
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i += kBatchSize_) {
+    const int batch_size = std::min(kBatchSize_, num_values - i);
+
+    for (int j = 0; j < batch_size; ++j) {
+      // Convert to ByteArray, so we can pass to the suffix_encoder_.
+      const ByteArray value = src[i + j];
+      if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+        throw ParquetException("Parquet cannot store strings with size 2GB or more");
+      }
+      auto view = std::string_view{value};
+
+      uint32_t k = 0;
+      const uint32_t common_length =
+          std::min(value.len, static_cast<uint32_t>(last_value_view.length()));
+      while (k < common_length) {
+        if (last_value_view[k] != view[k]) {
+          break;
+        }
+        k++;
+      }
+
+      last_value_view = view;
+      prefix_lengths_[j] = k;
+      const auto suffix_length = static_cast<uint32_t>(value.len - k);
+
+      if (suffix_length == 0) {
+        continue;
+      }
+      const uint8_t* suffix_ptr = value.ptr + k;
+      // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+      const ByteArray suffix(suffix_length, suffix_ptr);
+      suffix_encoder_.Put(&suffix, 1);
+    }
+    prefix_length_encoder_.Put(prefix_lengths_.data(), batch_size);

Review Comment:
   Added suffix batching (https://github.com/apache/arrow/pull/14341/commits/0766ee7ae9325ce08fb02c15d83023c60c8362f4). I'm not sure about performance either way.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1256735619


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,230 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();

Review Comment:
   Indeed!



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,230 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        auto view = visitor[i + j];

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1256700473


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1275,6 +1287,40 @@ struct ArrowBinaryHelper {
   int64_t chunk_space_remaining;
 };
 
+template <>
+struct ArrowBinaryHelper<FLBAType> {
+  explicit ArrowBinaryHelper(EncodingTraits<FLBAType>::Accumulator* builder) {
+    this->builder = builder;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 this->builder->value_data_length(),
+                                                 &this->chunk_space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<FLBAType>");
+    }
+  }
+
+  Status PushChunk() {
+    std::shared_ptr<::arrow::Array> result;
+    RETURN_NOT_OK(builder->Finish(&result));
+    chunks.push_back(std::move(result));
+    chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
+    return Status::OK();
+  }
+
+  bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
+
+  Status Append(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
+    chunk_space_remaining -= length;
+    return builder->Append(data);
+  }
+
+  Status AppendNull() { return builder->AppendNull(); }
+
+  ::arrow::FixedSizeBinaryBuilder* builder;
+  std::vector<std::shared_ptr<::arrow::Array>> chunks;

Review Comment:
   I don't think we discussed it. I've used your suggestion.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1256853063


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,14 +3058,237 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+  static constexpr std::string_view kEmpty = "";
+
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(descr, pool),

Review Comment:
   ```
     EncoderImpl(const ColumnDescriptor* descr, Encoding::type encoding, MemoryPool* pool)
         : descr_(descr),
           encoding_(encoding),
           pool_(pool),
           type_length_(descr ? descr->type_length() : -1) {}
   ```
   
   After checked, seems `descr` is for FLBA. I'm ok for using nullptr, and here we should add comment that why `prefix_length_encoder_` can has a null descr



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1263913094


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3058,235 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+  static constexpr std::string_view kEmpty = "";
+
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),

Review Comment:
   encoder like this would be ok, but could you add comment for why descr could be nullptr?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,43 +1240,62 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
-  explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
-    this->out = out;
-    this->builder = out->builder.get();
-    this->chunk_space_remaining =
-        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+template <typename DType, typename Enable = void>
+struct ArrowBinaryHelper;
+
+template <typename DType>
+struct ArrowBinaryHelper<DType, std::enable_if_t<std::is_same_v<DType, ByteArrayType> ||
+                                                     std::is_same_v<DType, FLBAType>,
+                                                 void>> {
+  explicit ArrowBinaryHelper(typename EncodingTraits<DType>::Accumulator* acc) {
+    builder = acc->builder.get();
+    chunks = acc->chunks;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 builder->value_data_length(),
+                                                 &chunk_space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<DType>");
+    }
   }
 
   Status PushChunk() {
     std::shared_ptr<::arrow::Array> result;
     RETURN_NOT_OK(builder->Finish(&result));
-    out->chunks.push_back(result);
+    chunks.push_back(std::move(result));
     chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
     return Status::OK();
   }
 
   bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
 
   void UnsafeAppend(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
     builder->UnsafeAppend(data, length);
   }
 
   void UnsafeAppendNull() { builder->UnsafeAppendNull(); }
 
-  Status Append(const uint8_t* data, int32_t length) {
-    chunk_space_remaining -= length;
-    return builder->Append(data, length);
-  }
+  virtual Status Append(const uint8_t* data, int32_t length);
 
   Status AppendNull() { return builder->AppendNull(); }
 
-  typename EncodingTraits<ByteArrayType>::Accumulator* out;
-  ::arrow::BinaryBuilder* builder;
+  typename EncodingTraits<DType>::BuilderType* builder;
+  std::vector<std::shared_ptr<::arrow::Array>> chunks;

Review Comment:
   This means that, when `PushChunk` called, only `chunks` pushback an element, but `builder->chunks` not. I guess using previous:
   
   ```c++
     typename EncodingTraits<ByteArrayType>::Accumulator* out;
     ::arrow::BinaryBuilder* builder;
   ```
   
   would be ok? Or
   
   ```
     typename EncodingTraits<DType>::BuilderType* builder;
     std::vector<std::shared_ptr<::arrow::Array>>* chunks;
   ```



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -874,7 +874,7 @@ std::shared_ptr<::arrow::Array> EncodingAdHocTyped<FLBAType>::GetValues(int seed
 }
 
 using EncodingAdHocTypedCases =
-    ::testing::Types<BooleanType, Int32Type, Int64Type, FloatType, DoubleType, FLBAType>;
+    ::testing::Types<BooleanType, Int32Type, Int64Type, FloatType, DoubleType>;

Review Comment:
   Just want to make clear why we modify the builder type? Maybe I've missed the context



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3058,235 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+  static constexpr std::string_view kEmpty = "";
+
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(descr, pool),
+        last_value_(""),
+        empty_(static_cast<uint32_t>(kEmpty.size()),
+               reinterpret_cast<const uint8_t*>(kEmpty.data())) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        const int idx = i + j;
+        auto view = visitor[idx];
+        len = visitor.len(idx);
+
+        uint32_t k = 0;
+        const uint32_t common_length =
+            std::min(len, static_cast<uint32_t>(last_value_view.length()));
+        while (k < common_length) {
+          if (last_value_view[k] != view[k]) {
+            break;
+          }
+          k++;
+        }
+
+        last_value_view = view;
+        prefix_lengths[j] = k;
+        const uint32_t suffix_length = len - k;
+        const uint8_t* suffix_ptr = src[idx].ptr + k;
+
+        // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+        const ByteArray suffix(suffix_length, suffix_ptr);
+        suffixes[j] = suffix;
+      }
+      suffix_encoder_.Put(suffixes.data(), batch_size);
+      prefix_length_encoder_.Put(prefix_lengths.data(), batch_size);
+    }
+    last_value_ = last_value_view;
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.length());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t len = src.len;
+          const uint32_t common_length = std::min(previous_len, len);

Review Comment:
   ditto, can we have better naming for `len` and `common_length`?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3058,235 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+  static constexpr std::string_view kEmpty = "";
+
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(descr, pool),
+        last_value_(""),
+        empty_(static_cast<uint32_t>(kEmpty.size()),
+               reinterpret_cast<const uint8_t*>(kEmpty.data())) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        const int idx = i + j;
+        auto view = visitor[idx];
+        len = visitor.len(idx);
+
+        uint32_t k = 0;
+        const uint32_t common_length =

Review Comment:
   I guess this is `common_length` or `maximum_common_prefix_length`, and `k` could be `common_prefix_length`?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3058,235 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+  static constexpr std::string_view kEmpty = "";
+
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(descr, pool),
+        last_value_(""),
+        empty_(static_cast<uint32_t>(kEmpty.size()),
+               reinterpret_cast<const uint8_t*>(kEmpty.data())) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        const int idx = i + j;
+        auto view = visitor[idx];
+        len = visitor.len(idx);

Review Comment:
   could we use `len` from `view`? like `view.len`?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3058,235 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+  static constexpr std::string_view kEmpty = "";
+
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(descr, pool),
+        last_value_(""),
+        empty_(static_cast<uint32_t>(kEmpty.size()),
+               reinterpret_cast<const uint8_t*>(kEmpty.data())) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();

Review Comment:
   Would you mind change `len` to `flba_len` to make the code a bit more clear



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1268249674


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3058,235 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+  static constexpr std::string_view kEmpty = "";
+
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),

Review Comment:
   ```suggestion
           prefix_length_encoder_(/*descr=*/nullptr, pool),
   ```
   
   I guess first using this is ok?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1269605891


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1908,4 +1907,304 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void InitData(int nvalues, int repeats, double prefixed_probability) {
+    num_values_ = nvalues * repeats;
+    input_bytes_.resize(num_values_ * sizeof(c_type));
+    output_bytes_.resize(num_values_ * sizeof(c_type));
+    draws_ = reinterpret_cast<c_type*>(input_bytes_.data());
+    decode_buf_ = reinterpret_cast<c_type*>(output_bytes_.data());
+    GeneratePrefixedData<c_type>(nvalues, draws_, &data_buffer_, prefixed_probability);
+
+    // add some repeated values
+    for (int j = 1; j < repeats; ++j) {
+      for (int i = 0; i < nvalues; ++i) {
+        draws_[nvalues * j + i] = draws_[i];
+      }
+    }
+  }
+
+  void Execute(int nvalues, int repeats, double prefixed_probability) {
+    InitData(nvalues, repeats, prefixed_probability);
+    CheckRoundtrip();
+  }
+
+  void ExecuteSpaced(int nvalues, int repeats, int64_t valid_bits_offset,

Review Comment:
   I've hardcoded it now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] wjones127 commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "wjones127 (via GitHub)" <gi...@apache.org>.
wjones127 commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1154873086


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          const uint8_t* suffix_ptr = src.ptr + j;
+          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+          last_value_view = view;
+          // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  int i = 0;
+  while (i < num_values) {
+    // Convert to ByteArray so we can pass to the suffix_encoder_.
+    auto value = reinterpret_cast<const ByteArray*>(&src[i]);
+    if (ARROW_PREDICT_FALSE(value->len >= kMaxByteArraySize)) {
+      throw Status::Invalid("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value->ptr), value->len};
+    uint32_t j = 0;
+    while (j < std::min(value->len, static_cast<uint32_t>(last_value_view.length()))) {

Review Comment:
   Ah I see now, `last_value_view` was being mutated in the loop. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1157449565


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          const uint8_t* suffix_ptr = src.ptr + j;
+          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+          last_value_view = view;
+          // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  int i = 0;
+  while (i < num_values) {
+    // Convert to ByteArray so we can pass to the suffix_encoder_.
+    auto value = reinterpret_cast<const ByteArray*>(&src[i]);
+    if (ARROW_PREDICT_FALSE(value->len >= kMaxByteArraySize)) {
+      throw Status::Invalid("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value->ptr), value->len};
+    uint32_t j = 0;
+    while (j < std::min(value->len, static_cast<uint32_t>(last_value_view.length()))) {

Review Comment:
   Indeed, changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1157448981


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          const uint8_t* suffix_ptr = src.ptr + j;
+          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+          last_value_view = view;
+          // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  int i = 0;
+  while (i < num_values) {
+    // Convert to ByteArray so we can pass to the suffix_encoder_.
+    auto value = reinterpret_cast<const ByteArray*>(&src[i]);

Review Comment:
   Added another Put for FLBA.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1157448112


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1160575251


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3196,6 +3467,43 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) {
+    return GetInternal(buffer, max_values);
+  }
+  int Decode(FixedLenByteArray* buffer, int max_values) override {
+    int decoded_values_size = max_values;
+    if (MultiplyWithOverflow(decoded_values_size,

Review Comment:
   Done.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,25 +1240,35 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
+template <typename DType>
+struct ArrowBinaryHelper;
+
+template <>
+struct ArrowBinaryHelper<ByteArrayType> {
   explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
     this->out = out;
     this->builder = out->builder.get();
+    if (SubtractWithOverflow(::arrow::kBinaryMemoryLimit,

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1159892879


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3083,242 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, nullptr)) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          buffer_, ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);

Review Comment:
   Switched to:
   ```cpp
   kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>("")))
   [...]
   if (suffix_length == 0) {
     suffix_encoder_.Put(&kEmpty, 1);
     return Status::OK();
   }
   ```
   and CI says it's ok. :)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1480568713

   (We can write a simple and slow one, and optimize it later?)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1147073137


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3064,269 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {

Review Comment:
   I vote for it, maybe I can try to extract it tonight :)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1121996557


##########
cpp/src/parquet/encoding.cc:
##########
@@ -2900,6 +2900,124 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+ public:
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr, MemoryPool* pool)
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY,
+                    pool = ::arrow::default_memory_pool()),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override {
+    AssertBaseBinary(values);
+    const auto& data = values.data();

Review Comment:
   Seems this can use the way `DeltaLength` encoder using



##########
cpp/src/parquet/encoding.cc:
##########
@@ -2900,6 +2900,124 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+ public:
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr, MemoryPool* pool)
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY,
+                    pool = ::arrow::default_memory_pool()),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override {
+    AssertBaseBinary(values);
+    const auto& data = values.data();
+    auto src = data->GetValues<T>(1);
+
+    if (values.null_count() == 0) {
+      Put(src, static_cast<int>(values.length()));
+    } else {
+      PutSpaced(src, static_cast<int>(data->length), data->GetValues<uint8_t>(0, 0),
+                data->offset);
+    }
+  }
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+  uint32_t total_value_count_{0};
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  string_view last_value_;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values);
+
+  if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+    last_value_ = string_view{reinterpret_cast<const char*>(src[0].ptr), src[0].len};
+    suffix_encoder_.Put(&src[0], 1);
+    prefix_lengths[0] = 0;
+  }
+  total_value_count_ += num_values;
+
+  for (int32_t i = 1; i < num_values; i++) {
+    auto value = string_view{reinterpret_cast<const char*>(src[i].ptr), src[i].len};
+
+    uint j = 0;
+    while (j < std::min(src[i - 1].len, src[i].len)) {
+      if (last_value_[j] != value[j]) {
+        break;
+      }
+      j++;
+    }
+
+    suffix_encoder_.Put(&src[i] + j, 1);

Review Comment:
   `&src[i] + j` lead to a wrong memory address. Seems that we should:
   
   ```
       ByteArray ba;
       ba.len = src[i].len - j;
       ba.ptr = src[i].ptr + j;
       suffix_encoder_.Put(&ba, 1);
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] github-actions[bot] commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1441873204

   * Closes: #32863


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] github-actions[bot] commented on pull request #14341: ARROW-17619: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1271095054

   https://issues.apache.org/jira/browse/ARROW-17619


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1269604665


##########
cpp/src/parquet/test_util.cc:
##########
@@ -132,5 +132,52 @@ void random_byte_array(int n, uint32_t seed, uint8_t* buf, ByteArray* out, int m
   random_byte_array(n, seed, buf, out, 0, max_size);
 }
 
+void prefixed_random_byte_array(int n, uint32_t seed, uint8_t* buf, ByteArray* out,
+                                int min_size, int max_size, double prefixed_probability) {
+  std::default_random_engine gen(seed);
+  std::uniform_int_distribution<int> d1(min_size, max_size);
+  std::uniform_int_distribution<int> d2(0, 255);
+  std::uniform_real_distribution<double> d3(0, 1);
+
+  for (int i = 0; i < n; ++i) {
+    int len = d1(gen);
+    out[i].len = len;
+    out[i].ptr = buf;
+
+    bool do_prefix = d3(gen) < prefixed_probability && i > 0;
+    std::uniform_int_distribution<int> d4(min_size, len);

Review Comment:
   Changed.



##########
cpp/src/parquet/test_util.h:
##########
@@ -783,8 +789,39 @@ inline void GenerateData<ByteArray>(int num_values, ByteArray* out,
   random_byte_array(num_values, 0, heap->data(), out, 2, max_byte_array_len);
 }
 
+template <typename T>
+inline void GeneratePrefixedData(int num_values, T* out, std::vector<uint8_t>* heap,
+                                 double prefixed_probability) {
+  // seed the prng so failure is deterministic

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1269606187


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1908,4 +1907,304 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void InitData(int nvalues, int repeats, double prefixed_probability) {
+    num_values_ = nvalues * repeats;
+    input_bytes_.resize(num_values_ * sizeof(c_type));
+    output_bytes_.resize(num_values_ * sizeof(c_type));
+    draws_ = reinterpret_cast<c_type*>(input_bytes_.data());
+    decode_buf_ = reinterpret_cast<c_type*>(output_bytes_.data());
+    GeneratePrefixedData<c_type>(nvalues, draws_, &data_buffer_, prefixed_probability);
+
+    // add some repeated values
+    for (int j = 1; j < repeats; ++j) {
+      for (int i = 0; i < nvalues; ++i) {
+        draws_[nvalues * j + i] = draws_[i];
+      }
+    }
+  }
+
+  void Execute(int nvalues, int repeats, double prefixed_probability) {
+    InitData(nvalues, repeats, prefixed_probability);
+    CheckRoundtrip();
+  }
+
+  void ExecuteSpaced(int nvalues, int repeats, int64_t valid_bits_offset,
+                     double null_probability, double prefixed_probability) {
+    InitData(nvalues, repeats, prefixed_probability);
+
+    int64_t size = num_values_ + valid_bits_offset;
+    auto rand = ::arrow::random::RandomArrayGenerator(1923);
+    const auto array = rand.UInt8(size, /*min=*/0, /*max=*/100, null_probability);
+    const auto valid_bits = array->null_bitmap_data();
+    if (valid_bits) {
+      CheckRoundtripSpaced(valid_bits, valid_bits_offset);
+    }
+  }
+
+  void CheckRoundtrip() override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+  std::vector<uint8_t> input_bytes_;
+  std::vector<uint8_t> output_bytes_;
+};
+
+using TestDeltaByteArrayEncodingTypes = ::testing::Types<ByteArrayType, FLBAType>;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, /*repeats=*/0, /*prefixed_probability=*/0.1));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 5, 0.2));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 1, 0.3));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_probability*/
+      0, 0.4));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.5, 0.5));
+}
+
+template <typename Type>
+class DeltaByteArrayEncodingDirectPut : public TestEncodingBase<Type> {
+ public:
+  std::unique_ptr<TypedEncoder<Type>> encoder =
+      MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+  std::unique_ptr<TypedDecoder<Type>> decoder =
+      MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+
+  void CheckDirectPut(std::shared_ptr<::arrow::Array> array) {
+    ASSERT_NO_THROW(encoder->Put(*array));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(array->length() - array->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<Type>::Accumulator acc;
+    using BuilderType = typename EncodingTraits<Type>::BuilderType;
+    acc.builder = std::make_unique<BuilderType>(array->type(), default_memory_pool());
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(array->length()),
+                                   static_cast<int>(array->null_count()),
+                                   array->null_bitmap_data(), array->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(array->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    ::arrow::AssertArraysEqual(*array, *result);
+  }
+
+  void CheckRoundtripFLBA() {
+    constexpr int64_t kSize = 50;
+    constexpr int kSeed = 42;
+    constexpr int kByteWidth = 4;
+    ::arrow::random::RandomArrayGenerator rag{kSeed};
+    std::shared_ptr<::arrow::Array> values =
+        rag.FixedSizeBinary(/*size=*/0, /*byte_width=*/kByteWidth);
+    CheckDirectPut(values);
+
+    for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+      rag = ::arrow::random::RandomArrayGenerator(seed);
+      values = rag.FixedSizeBinary(kSize + seed, kByteWidth);
+      CheckDirectPut(values);
+    }

Review Comment:
   Changed.



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1908,4 +1907,304 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void InitData(int nvalues, int repeats, double prefixed_probability) {
+    num_values_ = nvalues * repeats;
+    input_bytes_.resize(num_values_ * sizeof(c_type));
+    output_bytes_.resize(num_values_ * sizeof(c_type));
+    draws_ = reinterpret_cast<c_type*>(input_bytes_.data());
+    decode_buf_ = reinterpret_cast<c_type*>(output_bytes_.data());
+    GeneratePrefixedData<c_type>(nvalues, draws_, &data_buffer_, prefixed_probability);
+
+    // add some repeated values
+    for (int j = 1; j < repeats; ++j) {
+      for (int i = 0; i < nvalues; ++i) {
+        draws_[nvalues * j + i] = draws_[i];
+      }
+    }
+  }
+
+  void Execute(int nvalues, int repeats, double prefixed_probability) {
+    InitData(nvalues, repeats, prefixed_probability);
+    CheckRoundtrip();
+  }
+
+  void ExecuteSpaced(int nvalues, int repeats, int64_t valid_bits_offset,
+                     double null_probability, double prefixed_probability) {
+    InitData(nvalues, repeats, prefixed_probability);
+
+    int64_t size = num_values_ + valid_bits_offset;
+    auto rand = ::arrow::random::RandomArrayGenerator(1923);
+    const auto array = rand.UInt8(size, /*min=*/0, /*max=*/100, null_probability);
+    const auto valid_bits = array->null_bitmap_data();
+    if (valid_bits) {
+      CheckRoundtripSpaced(valid_bits, valid_bits_offset);
+    }
+  }
+
+  void CheckRoundtrip() override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+  std::vector<uint8_t> input_bytes_;
+  std::vector<uint8_t> output_bytes_;
+};
+
+using TestDeltaByteArrayEncodingTypes = ::testing::Types<ByteArrayType, FLBAType>;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, /*repeats=*/0, /*prefixed_probability=*/0.1));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 5, 0.2));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 1, 0.3));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_probability*/
+      0, 0.4));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.5, 0.5));
+}
+
+template <typename Type>
+class DeltaByteArrayEncodingDirectPut : public TestEncodingBase<Type> {
+ public:
+  std::unique_ptr<TypedEncoder<Type>> encoder =
+      MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+  std::unique_ptr<TypedDecoder<Type>> decoder =
+      MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+
+  void CheckDirectPut(std::shared_ptr<::arrow::Array> array) {
+    ASSERT_NO_THROW(encoder->Put(*array));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(array->length() - array->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<Type>::Accumulator acc;
+    using BuilderType = typename EncodingTraits<Type>::BuilderType;
+    acc.builder = std::make_unique<BuilderType>(array->type(), default_memory_pool());
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(array->length()),
+                                   static_cast<int>(array->null_count()),
+                                   array->null_bitmap_data(), array->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(array->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    ::arrow::AssertArraysEqual(*array, *result);
+  }
+
+  void CheckRoundtripFLBA() {
+    constexpr int64_t kSize = 50;
+    constexpr int kSeed = 42;
+    constexpr int kByteWidth = 4;
+    ::arrow::random::RandomArrayGenerator rag{kSeed};
+    std::shared_ptr<::arrow::Array> values =
+        rag.FixedSizeBinary(/*size=*/0, /*byte_width=*/kByteWidth);
+    CheckDirectPut(values);
+
+    for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+      rag = ::arrow::random::RandomArrayGenerator(seed);
+      values = rag.FixedSizeBinary(kSize + seed, kByteWidth);
+      CheckDirectPut(values);
+    }
+  }
+
+  void CheckRoundtripByteArray() {
+    constexpr int64_t kSize = 500;
+    constexpr int32_t kMinLength = 0;
+    constexpr int32_t kMaxLength = 10;
+    constexpr int32_t kNumUnique = 10;
+    constexpr double kNullProbability = 0.25;
+    constexpr int kSeed = 42;
+    ::arrow::random::RandomArrayGenerator rag{kSeed};
+    std::shared_ptr<::arrow::Array> values = rag.BinaryWithRepeats(
+        /*size=*/1, /*unique=*/1, kMinLength, kMaxLength, kNullProbability);
+    CheckDirectPut(values);
+
+    for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+      rag = ::arrow::random::RandomArrayGenerator(seed);
+      values = rag.BinaryWithRepeats(kSize, kNumUnique, kMinLength, kMaxLength,
+                                     kNullProbability);
+      CheckDirectPut(values);
+    }
+  }
+
+  void CheckRoundtrip() override {
+    using ArrowType = typename EncodingTraits<Type>::ArrowType;
+    using IsFixedSizeBinary = ::arrow::is_fixed_size_binary_type<ArrowType>;
+
+    if constexpr (IsFixedSizeBinary::value) {
+      CheckRoundtripFLBA();
+    } else {
+      CheckRoundtripByteArray();
+    }
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+TYPED_TEST_SUITE(DeltaByteArrayEncodingDirectPut, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(DeltaByteArrayEncodingDirectPut, DirectPut) {
+  ASSERT_NO_FATAL_FAILURE(this->CheckRoundtrip());
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
+  auto CheckEncode = [](const std::shared_ptr<::arrow::Array>& values,
+                        const std::shared_ptr<Buffer>& encoded) {
+    auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+    ASSERT_TRUE(encoded->Equals(*buf));
+  };
+
+  auto ArrayToInt32Vector = [](const std::shared_ptr<::arrow::Array>& lengths) {
+    std::vector<int32_t> arrays;

Review Comment:
   Changed to vector.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1269606757


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,43 +1240,62 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
-  explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
-    this->out = out;
-    this->builder = out->builder.get();
-    this->chunk_space_remaining =
-        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+template <typename DType, typename Enable = void>
+struct ArrowBinaryHelper;
+
+template <typename DType>
+struct ArrowBinaryHelper<DType, std::enable_if_t<std::is_same_v<DType, ByteArrayType> ||
+                                                     std::is_same_v<DType, FLBAType>,
+                                                 void>> {
+  explicit ArrowBinaryHelper(typename EncodingTraits<DType>::Accumulator* acc) {
+    builder = acc->builder.get();
+    chunks = &acc->chunks;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 builder->value_data_length(),
+                                                 &chunk_space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<DType>");
+    }
   }
 
   Status PushChunk() {
     std::shared_ptr<::arrow::Array> result;
     RETURN_NOT_OK(builder->Finish(&result));
-    out->chunks.push_back(result);
+    chunks->push_back(std::move(result));
     chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
     return Status::OK();
   }
 
   bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
 
   void UnsafeAppend(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
     builder->UnsafeAppend(data, length);
   }
 
   void UnsafeAppendNull() { builder->UnsafeAppendNull(); }
 
-  Status Append(const uint8_t* data, int32_t length) {
-    chunk_space_remaining -= length;
-    return builder->Append(data, length);
-  }
+  virtual Status Append(const uint8_t* data, int32_t length);

Review Comment:
   Removed `virtual`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1269625032


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,43 +1240,62 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
-  explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
-    this->out = out;
-    this->builder = out->builder.get();
-    this->chunk_space_remaining =
-        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+template <typename DType, typename Enable = void>
+struct ArrowBinaryHelper;
+
+template <typename DType>
+struct ArrowBinaryHelper<DType, std::enable_if_t<std::is_same_v<DType, ByteArrayType> ||
+                                                     std::is_same_v<DType, FLBAType>,
+                                                 void>> {
+  explicit ArrowBinaryHelper(typename EncodingTraits<DType>::Accumulator* acc) {
+    builder = acc->builder.get();
+    chunks = &acc->chunks;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 builder->value_data_length(),
+                                                 &chunk_space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<DType>");
+    }
   }
 
   Status PushChunk() {
     std::shared_ptr<::arrow::Array> result;
     RETURN_NOT_OK(builder->Finish(&result));
-    out->chunks.push_back(result);
+    chunks->push_back(std::move(result));
     chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
     return Status::OK();
   }
 
   bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
 
   void UnsafeAppend(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));

Review Comment:
   `Append` is templated to allow for different `Append`s while we only seem to use `UnsafeAppend` for `ByteArray` type. Made everything templated and added `DCHECK(CanFit(length))`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1530087493

   Do we need another review here?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1550607525

   @rok I've written a bit ugly random string generator:
   
   ```C++
   std::shared_ptr<::arrow::Array> generateShareSeqString(
       ::arrow::random::RandomArrayGenerator& gen, size_t seqNum, double nullPercent) {
     ::arrow::StringBuilder builder;
     auto seed = gen.seed();
     std::shared_ptr<::arrow::Buffer> nullBitMap = nullptr;
     if (nullPercent != 0) {
       nullBitMap = gen.NullBitmap(seqNum, nullPercent);
     }
     std::mt19937 engine(seed);
     std::uniform_int_distribution<int32_t> igen;
     std::string s = "share-seq-string";
     std::string prefix = s;
   
     for (size_t i = 0; i < seqNum; ++i) {
       if (nullBitMap != nullptr && ::arrow::bit_util::GetBit(nullBitMap->data(), i)) {
         PARQUET_THROW_NOT_OK(builder.AppendNull());
       } else {
         PARQUET_THROW_NOT_OK(builder.Append(s));
         int length = igen(engine) % 4 + 1;  // random string length between 1 and 4
         std::string randomStr;
         for (int j = 0; j < length; j++) {
           char c = 'a' + igen(engine) % 26;  // random lowercase letter
           s += c;
         }
         s = prefix + randomStr;
         if (igen(engine) % 2 == 0) {  // randomly change prefix
           prefix = s.substr(0, igen(engine) % s.length());
         }
       }
     }
     PARQUET_ASSIGN_OR_THROW(auto array, builder.Finish());
     if (static_cast<size_t>(array->length()) != seqNum) {
       throw ParquetException("Error");
     }
     return array;
   }
   ```
   
   Maybe you can change a bit and use it


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1196439805


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,243 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw ParquetException("Put not implemented for " + this->descr_->ToString());
+}
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i++) {
+    // Convert to ByteArray, so we can pass to the suffix_encoder_.
+    const ByteArray value = src[i];
+    if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+      throw ParquetException("Parquet cannot store strings with size 2GB or more");
+    }
+
+    auto view = string_view{reinterpret_cast<const char*>(value.ptr),
+                            static_cast<uint32_t>(value.len)};
+    uint32_t j = 0;
+    const uint32_t common_length =
+        std::min(value.len, static_cast<uint32_t>(last_value_view.length()));
+    while (j < common_length) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
+    last_value_view = view;
+    prefix_lengths[i] = j;
+    const auto suffix_length = static_cast<uint32_t>(value.len - j);
+
+    if (suffix_length == 0) {
+      suffix_encoder_.Put(&kEmpty, 1);
+      continue;
+    }
+    const uint8_t* suffix_ptr = value.ptr + j;
+    // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+    const ByteArray suffix(suffix_length, suffix_ptr);
+    suffix_encoder_.Put(&suffix, 1);
+  }
+  prefix_length_encoder_.Put(prefix_lengths.data(), num_values);
+  last_value_ = last_value_view;
+}
+
+template <>
+void DeltaByteArrayEncoder<FLBAType>::Put(const FLBA* src, int num_values) {

Review Comment:
   I'm not sure about "hard". It could be something like:
   ```c++
   struct ByteArrayVisitor {
     const ByteArray* src;
   
     std::string_view operator[](int i) const {
       return std::string_view(src[i]);
     }
   };
   struct FLBAVisitor {
     const FLBA* src;
     int32_t len;
   
     std::string_view operator[](int i) const {
       return std::string_view(src[i].ptr, len);
     }
   };
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1198455430


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1955,5 +1955,293 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void CheckRoundtrip() override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder =
+        MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY, false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+typedef ::testing::Types<ByteArrayType, FLBAType> TestDeltaByteArrayEncodingTypes;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  ASSERT_NO_FATAL_FAILURE(this->Execute(250, 2));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_prob*/ 0));
+
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, 200));
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+      /*null_probability*/ 0.1));
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowBinaryDirectPut) {
+  const int64_t size = 50;
+  const int32_t min_length = 0;
+  const int32_t max_length = 10;
+  const int32_t num_unique = 10;
+  const double null_probability = 0.25;
+  auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+  auto decoder = MakeTypedDecoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+
+  auto CheckSeed = [&](std::shared_ptr<::arrow::Array> values) {
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(values->length() - values->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<ByteArrayType>::Accumulator acc;
+    if (::arrow::is_string(values->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(values->length()),
+                                   static_cast<int>(values->null_count()),
+                                   values->null_bitmap_data(), values->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(values->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    auto upcast_result = CastBinaryTypesHelper(result, values->type());
+    ::arrow::AssertArraysEqual(*values, *result);
+  };
+
+  ::arrow::random::RandomArrayGenerator rag(42);
+  auto values = rag.String(0, min_length, max_length, null_probability);

Review Comment:
   See https://github.com/apache/arrow/pull/14341#discussion_r1197178782.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1142865848


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3265,6 +3420,14 @@ std::unique_ptr<Encoder> MakeEncoder(Type::type type_num, Encoding::type encodin
       default:
         throw ParquetException("DELTA_LENGTH_BYTE_ARRAY only supports BYTE_ARRAY");
     }
+  } else if (encoding == Encoding::DELTA_BYTE_ARRAY) {
+    switch (type_num) {
+      case Type::BYTE_ARRAY:
+        return std::make_unique<DeltaByteArrayEncoder<ByteArrayType>>(descr, pool);
+      default:
+        throw ParquetException("DELTA_BYTE_ARRAY only supports BYTE_ARRAY");

Review Comment:
   Added! 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1144171836


##########
cpp/src/parquet/encoding.cc:
##########
@@ -2900,8 +2929,222 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+ public:
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr, MemoryPool* pool)
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY,
+                    pool = ::arrow::default_memory_pool()),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+
+    // TODO(rok): optimize using ArrowPoolVector<int32_t> prefix_lengths(num_values);
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          uint32_t previous_len = 0;

Review Comment:
   :facepalm: yes indeed. Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1503060935

   I mean a more thorough review to check the implementation for potential issues.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1162181750


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3083,242 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          buffer_, ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));

Review Comment:
   Removed as a member. I think the idea was to save time creating it every time `PutSpaced` is called.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1157448630


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert view to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          while (j < std::min(previous_len, src.len)) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          const uint8_t* suffix_ptr = src.ptr + j;
+          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+          last_value_view = view;
+          // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);

Review Comment:
   Not sure. Added a check to avoid it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1158125248


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3177,8 +3194,7 @@ void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
                                           ::arrow::stl::allocator<int32_t>(pool_));
   std::string_view last_value_view = last_value_;
 
-  int i = 0;
-  while (i < num_values) {
+  for (int i = 0; i < num_values; i++) {

Review Comment:
   Added generic function that throws and changed the comments a little bit.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3187,21 +3203,74 @@ void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
 
     auto view = string_view{reinterpret_cast<const char*>(value->ptr), value->len};
     uint32_t j = 0;
-    while (j < std::min(value->len, static_cast<uint32_t>(last_value_view.length()))) {
+    const uint32_t common_length =
+        std::min(value->len, static_cast<uint32_t>(last_value_view.length()));
+    while (j < common_length) {
       if (last_value_view[j] != view[j]) {
         break;
       }
       j++;
     }
 
+    last_value_view = view;
     prefix_lengths[i] = j;
-    const uint8_t* suffix_ptr = value->ptr + j;
-    const uint32_t suffix_length = static_cast<uint32_t>(value->len - j);
+    const auto suffix_length = static_cast<uint32_t>(value->len - j);
+    const uint8_t* suffix_ptr;
+    if (suffix_length == 0) {
+      suffix_ptr = reinterpret_cast<const uint8_t*>("");

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] wgtmac commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1497386896

   The ASAN UBSAN check is failed: https://github.com/apache/arrow/actions/runs/4616021690/jobs/8160523964?pr=14341
   
   ```
   [----------] 3 tests from DeltaByteArrayEncodingAdHoc
   [ RUN      ] DeltaByteArrayEncodingAdHoc.ArrowBinaryDirectPut
   /arrow/cpp/src/arrow/buffer_builder.h:138:27: runtime error: null pointer passed as argument 2, which is declared to never be null
   /usr/include/string.h:44:28: note: nonnull attribute specified here
       #0 0x559f81bab658 in arrow::BufferBuilder::UnsafeAppend(void const*, long) /arrow/cpp/src/arrow/buffer_builder.h:138:5
       #1 0x7fecaef96580 in parquet::(anonymous namespace)::DeltaLengthByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::Put(parquet::ByteArray const*, int) /arrow/cpp/src/parquet/encoding.cc:2719:11
       #2 0x7fecaefc4218 in void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'(std::basic_string_view<char, std::char_traits<char> >)::operator()(std::basic_string_view<char, std::char_traits<char> >) const /arrow/cpp/src/parquet/encoding.cc:3149:5
       #3 0x7fecaefc3046 in arrow::Status arrow::internal::ArraySpanInlineVisitor<arrow::BinaryType, void>::VisitStatus<void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'(std::basic_string_view<char, std::char_traits<char> >), void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'()>(arrow::ArraySpan const&, arrow::BinaryArray&&, void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'()&&)::'lambda'(long)::operator()(long) const /arrow/cpp/src/arrow/visit_data_inline.h:109:18
       #4 0x7fecaefc2090 in arrow::Status arrow::internal::VisitBitBlocks<arrow::Status arrow::internal::ArraySpanInlineVisitor<arrow::BinaryType, void>::VisitStatus<void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'(std::basic_string_view<char, std::char_traits<char> >), void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'()>(arrow::ArraySpan const&, arrow::BinaryArray&&, void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'()&&)::'lambda'(long), arrow::Status arrow::internal::ArraySpanInlineVisitor<arrow::BinaryType, void>::VisitStatus<void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::Phys
 icalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'(std::basic_string_view<char, std::char_traits<char> >), void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'()>(arrow::ArraySpan const&, arrow::BinaryArray&&, void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'()&&)::'lambda'()>(unsigned char const*, long, long, arrow::BinaryArray&&, void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'()&&) /arrow/cpp/src/arrow/util/bit_block_counter.h:445:11
       #5 0x7fecaefc1072 in arrow::Status arrow::internal::ArraySpanInlineVisitor<arrow::BinaryType, void>::VisitStatus<void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'(std::basic_string_view<char, std::char_traits<char> >), void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'()>(arrow::ArraySpan const&, arrow::BinaryArray&&, void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'()&&) /arrow/cpp/src/arrow/visit_data_inline.h:103:12
       #6 0x7fecaefc0415 in std::enable_if<std::is_same<decltype(return_type_impl(&(std::decay<void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'(std::basic_string_view<char, std::char_traits<char> >)>::type::operator()))), arrow::Status>::value, arrow::Status>::type arrow::VisitArraySpanInline<arrow::BinaryType, void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'(std::basic_string_view<char, std::char_traits<char> >), void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'()>(arrow::ArraySpan const&, void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::Pu
 tBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'(std::basic_string_view<char, std::char_traits<char> >)&&, void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&)::'lambda'()&&) /arrow/cpp/src/arrow/visit_data_inline.h:195:10
       #7 0x7fecaefbe744 in void parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::PutBinaryArray<arrow::BinaryArray>(arrow::BinaryArray const&) /arrow/cpp/src/parquet/encoding.cc:3149:5
       #8 0x7fecaefbb3b4 in parquet::(anonymous namespace)::DeltaByteArrayEncoder<parquet::PhysicalType<(parquet::Type::type)6> >::Put(arrow::Array const&) /arrow/cpp/src/parquet/encoding.cc:3293:5
       #9 0x559f81b7f9b4 in parquet::test::DeltaByteArrayEncodingAdHoc_ArrowBinaryDirectPut_Test::TestBody()::$_5::operator()(std::shared_ptr<arrow::Array>) const /arrow/cpp/src/parquet/encoding_test.cc:2035:5
       #10 0x559f81b7ec85 in parquet::test::DeltaByteArrayEncodingAdHoc_ArrowBinaryDirectPut_Test::TestBody() /arrow/cpp/src/parquet/encoding_test.cc:2068:5
       #11 0x7fecb0bf85aa in void testing::internal::HandleSehExceptionsInMethodIfSupported<testing::Test, void>(testing::Test*, void (testing::Test::*)(), char const*) /build/cpp/googletest_ep-prefix/src/googletest_ep/googletest/src/gtest.cc:2607:10
       #12 0x7fecb0bdcc19 in void testing::internal::HandleExceptionsInMethodIfSupported<testing::Test, void>(testing::Test*, void (testing::Test::*)(), char const*) /build/cpp/googletest_ep-prefix/src/googletest_ep/googletest/src/gtest.cc:2643:14
       #13 0x7fecb0bb6a82 in testing::Test::Run() /build/cpp/googletest_ep-prefix/src/googletest_ep/googletest/src/gtest.cc:2682:5
       #14 0x7fecb0bb77e8 in testing::TestInfo::Run() /build/cpp/googletest_ep-prefix/src/googletest_ep/googletest/src/gtest.cc:2861:11
       #15 0x7fecb0bb8003 in testing::TestSuite::Run() /build/cpp/googletest_ep-prefix/src/googletest_ep/googletest/src/gtest.cc:3015:28
       #16 0x7fecb0bc8981 in testing::internal::UnitTestImpl::RunAllTests() /build/cpp/googletest_ep-prefix/src/googletest_ep/googletest/src/gtest.cc:5855:44
       #17 0x7fecb0bfb5aa in bool testing::internal::HandleSehExceptionsInMethodIfSupported<testing::internal::UnitTestImpl, bool>(testing::internal::UnitTestImpl*, bool (testing::internal::UnitTestImpl::*)(), char const*) /build/cpp/googletest_ep-prefix/src/googletest_ep/googletest/src/gtest.cc:2607:10
       #18 0x7fecb0bdf419 in bool testing::internal::HandleExceptionsInMethodIfSupported<testing::internal::UnitTestImpl, bool>(testing::internal::UnitTestImpl*, bool (testing::internal::UnitTestImpl::*)(), char const*) /build/cpp/googletest_ep-prefix/src/googletest_ep/googletest/src/gtest.cc:2643:14
       #19 0x7fecb0bc84ea in testing::UnitTest::Run() /build/cpp/googletest_ep-prefix/src/googletest_ep/googletest/src/gtest.cc:5438:10
       #20 0x7fecb0c32210 in RUN_ALL_TESTS() /build/cpp/googletest_ep-prefix/src/googletest_ep/googletest/include/gtest/gtest.h:2490:46
       #21 0x7fecb0c321ec in main /build/cpp/googletest_ep-prefix/src/googletest_ep/googletest/src/gtest_main.cc:52:10
       #0 0x7fec91e42d8f in
       #23 0x7fec91e42e3f in __libc_start_main (/lib/x86_64-linux-gnu/libc.so.6+0x29e3f) (BuildId: 69389d485a9793dbe873f0ea2c93e02efaa9aa3d)
       #24 0x559f817bb0d4 in _start (/build/cpp/debug/parquet-internals-test+0x5bc0d4) (BuildId: 4c7595fb7dab36a6ca4c879fa4ce83f9ed7a537e)
   
   SUMMARY: UndefinedBehaviorSanitizer: undefined-behavior /arrow/cpp/src/arrow/buffer_builder.h:138:27 in
   /build/cpp/src/parquet
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] wgtmac commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1157921371


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3149,9 +3160,14 @@ class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DT
           previous_len = src.len;
           prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
 
-          const uint8_t* suffix_ptr = src.ptr + j;
-          const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
           last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          const uint8_t* suffix_ptr;
+          if (suffix_length == 0) {
+            suffix_ptr = reinterpret_cast<const uint8_t*>("");

Review Comment:
   `""` is a temporary variable on stack which only lives in the current scope. What about `suffix_ptr = nullptr` or use a `static const std::string kEmpty = ""`?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3187,21 +3203,74 @@ void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
 
     auto view = string_view{reinterpret_cast<const char*>(value->ptr), value->len};
     uint32_t j = 0;
-    while (j < std::min(value->len, static_cast<uint32_t>(last_value_view.length()))) {
+    const uint32_t common_length =
+        std::min(value->len, static_cast<uint32_t>(last_value_view.length()));
+    while (j < common_length) {
       if (last_value_view[j] != view[j]) {
         break;
       }
       j++;
     }
 
+    last_value_view = view;
     prefix_lengths[i] = j;
-    const uint8_t* suffix_ptr = value->ptr + j;
-    const uint32_t suffix_length = static_cast<uint32_t>(value->len - j);
+    const auto suffix_length = static_cast<uint32_t>(value->len - j);
+    const uint8_t* suffix_ptr;
+    if (suffix_length == 0) {
+      suffix_ptr = reinterpret_cast<const uint8_t*>("");
+    } else {
+      suffix_ptr = value->ptr + j;
+    }
+    // Convert suffix to ByteArray so it can be passed to the suffix_encoder_.
+    const ByteArray suffix(suffix_length, suffix_ptr);
+    suffix_encoder_.Put(&suffix, 1);
+  }
+  prefix_length_encoder_.Put(prefix_lengths.data(), num_values);
+  last_value_ = last_value_view;
+}
+
+template <>
+void DeltaByteArrayEncoder<FLBAType>::Put(const FLBA* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+  const int32_t len = descr_->type_length();
+
+  if (ARROW_PREDICT_FALSE(len >= static_cast<int32_t>(kMaxByteArraySize))) {
+    throw Status::Invalid("Parquet cannot store strings with size 2GB or more");
+  }
+
+  for (int i = 0; i < num_values; i++) {
+    // Convert to ByteArray so we can pass to the suffix_encoder_.
+    const FLBA* value = reinterpret_cast<const FLBA*>(&src[i].ptr);
+
+    auto view = string_view{reinterpret_cast<const char*>(value->ptr),
+                            static_cast<uint32_t>(len)};
+    int32_t j = 0;
+    const int32_t common_length =
+        std::min(len, static_cast<int32_t>(last_value_view.length()));
+    while (j < common_length) {
+      if (last_value_view[j] != view[j]) {
+        break;
+      }
+      j++;
+    }
+
     last_value_view = view;
+    prefix_lengths[i] = j;
+    const auto suffix_length = static_cast<uint32_t>(len - j);
+    const uint8_t* suffix_ptr;
+    if (suffix_length == 0) {
+      suffix_ptr = reinterpret_cast<const uint8_t*>("");

Review Comment:
   ditto



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3177,8 +3194,7 @@ void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
                                           ::arrow::stl::allocator<int32_t>(pool_));
   std::string_view last_value_view = last_value_;
 
-  int i = 0;
-  while (i < num_values) {
+  for (int i = 0; i < num_values; i++) {

Review Comment:
   nit: what about make the current function a specialization for ByteArrayType and throw an unimplemented exception in the generic one? This may make the line 3199 less confusing.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3187,21 +3203,74 @@ void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
 
     auto view = string_view{reinterpret_cast<const char*>(value->ptr), value->len};
     uint32_t j = 0;
-    while (j < std::min(value->len, static_cast<uint32_t>(last_value_view.length()))) {
+    const uint32_t common_length =
+        std::min(value->len, static_cast<uint32_t>(last_value_view.length()));
+    while (j < common_length) {
       if (last_value_view[j] != view[j]) {
         break;
       }
       j++;
     }
 
+    last_value_view = view;
     prefix_lengths[i] = j;
-    const uint8_t* suffix_ptr = value->ptr + j;
-    const uint32_t suffix_length = static_cast<uint32_t>(value->len - j);
+    const auto suffix_length = static_cast<uint32_t>(value->len - j);
+    const uint8_t* suffix_ptr;
+    if (suffix_length == 0) {
+      suffix_ptr = reinterpret_cast<const uint8_t*>("");

Review Comment:
   ditto. suffix_ptr = nullptr would be the simplest fix.



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -2224,15 +2223,24 @@ TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
     auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([6, 0, 0, 0])");
 
     constexpr std::string_view suffix_data = "axisba";
-    checkEncodeDecode(values, prefix_lengths, suffix_lengths, suffix_data);
+    CheckEncodeDecode(values, prefix_lengths, suffix_lengths, suffix_data);
   }
+
   {
     auto values = R"(["baaxis", "axis", "axis", "axis"])";
     auto prefix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([0, 0, 4, 4])");
     auto suffix_lengths = ::arrow::ArrayFromJSON(::arrow::int32(), R"([6, 4, 0, 0])");
 
     constexpr std::string_view suffix_data = "baaxisaxis";
-    checkEncodeDecode(values, prefix_lengths, suffix_lengths, suffix_data);
+    CheckEncodeDecode(values, prefix_lengths, suffix_lengths, suffix_data);
+  }
+
+  {
+    auto values = R"(["καλημέρα", "καμηλιέρη", "καμηλιέρη", "καλημέρα"])";

Review Comment:
   Thanks for adding this!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1155599136


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");

Review Comment:
   We have [this discussion](https://issues.apache.org/jira/browse/ARROW-10426?focusedCommentId=17223386&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17223386).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1147030620


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1275,6 +1275,35 @@ struct ArrowBinaryHelper {
   int64_t chunk_space_remaining;
 };
 
+struct ArrowFLBAHelper {
+  explicit ArrowFLBAHelper(::arrow::FixedSizeBinaryBuilder* builder) {
+    this->builder = builder;
+    this->chunk_space_remaining =
+        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+  }
+
+  Status PushChunk() {
+    std::shared_ptr<::arrow::Array> result;
+    RETURN_NOT_OK(builder->Finish(&result));
+    chunks.push_back(result);

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1148415819


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,180 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;

Review Comment:
   Huh, that throws `"prefix length too large in DELTA_BYTE_ARRAY"`. I'll take a look at it tomorrow.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1145648518


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3062,226 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+    uint32_t previous_len = byte_width;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = byte_width;
+          } else {
+            uint32_t j = 0;
+            while (j < previous_len) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(byte_width - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_TRUE(view.size() > kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = src.len;
+          } else {
+            uint32_t j = 0;
+            while (j < std::min(previous_len, src.len)) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  string_view last_value_;
+};
+
+template <>
+inline void DeltaByteArrayEncoder<FLBAType>::Put(const FixedLenByteArray* src,
+                                                 int num_values) {
+  if (descr_->type_length() == 0) {
+    return;
+  }
+  for (int i = 0; i < num_values; ++i) {
+    // Write the result to the output stream
+    DCHECK(src[i].ptr != nullptr) << "Value ptr cannot be NULL";
+    PARQUET_THROW_NOT_OK(sink_.Append(src[i].ptr, descr_->type_length()));

Review Comment:
   Seems that this was not `DeltaByteArray`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1146208807


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3062,226 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+    uint32_t previous_len = byte_width;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = byte_width;
+          } else {
+            uint32_t j = 0;
+            while (j < previous_len) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(byte_width - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_TRUE(view.size() > kMaxByteArraySize)) {

Review Comment:
   Switched to `>=`. Can FLBA actually be too big?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1146206553


##########
cpp/src/parquet/encoding.cc:
##########
@@ -2900,8 +2929,222 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+ public:
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr, MemoryPool* pool)
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY,
+                    pool = ::arrow::default_memory_pool()),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+
+    // TODO(rok): optimize using ArrowPoolVector<int32_t> prefix_lengths(num_values);
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          uint32_t previous_len = 0;
+          const ByteArray src{view};

Review Comment:
   Added a short comment to the FLBA instance.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1686708246

   I'm going to merge this now.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1295911452


##########
cpp/src/parquet/test_util.cc:
##########
@@ -132,5 +133,54 @@ void random_byte_array(int n, uint32_t seed, uint8_t* buf, ByteArray* out, int m
   random_byte_array(n, seed, buf, out, 0, max_size);
 }
 
+void prefixed_random_byte_array(int n, uint32_t seed, uint8_t* buf, ByteArray* out,
+                                int min_size, int max_size, double prefixed_probability) {
+  std::default_random_engine gen(seed);
+  std::uniform_int_distribution<int> dist_size(min_size, max_size);
+  std::uniform_int_distribution<int> dist_byte(0, 255);
+  std::bernoulli_distribution dist_has_prefix(prefixed_probability);
+  std::uniform_real_distribution<double> dist_prefix_length(0, 1);
+
+  for (int i = 0; i < n; ++i) {
+    int len = dist_size(gen);
+    out[i].len = len;
+    out[i].ptr = buf;
+
+    bool do_prefix = dist_has_prefix(gen) && i > 0;
+    std::uniform_int_distribution<int> d4(min_size, len);

Review Comment:
   `d4` can be removed now that it's unused.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1299967250


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,43 +1240,93 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
-  explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
-    this->out = out;
-    this->builder = out->builder.get();
-    this->chunk_space_remaining =
-        ::arrow::kBinaryMemoryLimit - this->builder->value_data_length();
+template <typename DType, typename Enable = void>
+struct ArrowBinaryHelper;
+
+template <typename DType>
+struct ArrowBinaryHelper<DType,
+                         std::enable_if_t<std::is_same_v<DType, ByteArrayType>, void>> {
+  explicit ArrowBinaryHelper(typename EncodingTraits<DType>::Accumulator* acc) {
+    builder = acc->builder.get();
+    chunks = &acc->chunks;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 builder->value_data_length(),
+                                                 &chunk_space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<DType>");
+    }
   }
 
   Status PushChunk() {
     std::shared_ptr<::arrow::Array> result;
     RETURN_NOT_OK(builder->Finish(&result));
-    out->chunks.push_back(result);
+    chunks->push_back(result);
     chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
     return Status::OK();
   }
 
   bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
 
   void UnsafeAppend(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
     builder->UnsafeAppend(data, length);
   }
 
   void UnsafeAppendNull() { builder->UnsafeAppendNull(); }
 
   Status Append(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
     chunk_space_remaining -= length;
     return builder->Append(data, length);
   }
 
   Status AppendNull() { return builder->AppendNull(); }
 
-  typename EncodingTraits<ByteArrayType>::Accumulator* out;
-  ::arrow::BinaryBuilder* builder;
+  typename EncodingTraits<DType>::BuilderType* builder;
+  std::vector<std::shared_ptr<::arrow::Array>>* chunks;
   int64_t chunk_space_remaining;
 };
 
+template <typename DType>
+struct ArrowBinaryHelper<DType, std::enable_if_t<std::is_same_v<DType, FLBAType>, void>> {
+  explicit ArrowBinaryHelper(typename EncodingTraits<DType>::Accumulator* acc) {
+    builder = acc;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 builder->value_data_length(),
+                                                 &space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<DType>");
+    }
+  }
+
+  Status PushChunk() {
+    std::shared_ptr<::arrow::Array> result;
+    RETURN_NOT_OK(builder->Finish(&result));

Review Comment:
   Indeed this would not make sense. I've pushed a change for `ArrowBinaryHelper<FLBAType>` to have an internal `std::vector<std::shared_ptr<::arrow::Array>>* chunks` and store results there.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1297147443


##########
cpp/src/parquet/test_util.cc:
##########
@@ -132,5 +133,54 @@ void random_byte_array(int n, uint32_t seed, uint8_t* buf, ByteArray* out, int m
   random_byte_array(n, seed, buf, out, 0, max_size);
 }
 
+void prefixed_random_byte_array(int n, uint32_t seed, uint8_t* buf, ByteArray* out,
+                                int min_size, int max_size, double prefixed_probability) {
+  std::default_random_engine gen(seed);
+  std::uniform_int_distribution<int> dist_size(min_size, max_size);
+  std::uniform_int_distribution<int> dist_byte(0, 255);
+  std::bernoulli_distribution dist_has_prefix(prefixed_probability);
+  std::uniform_real_distribution<double> dist_prefix_length(0, 1);
+
+  for (int i = 0; i < n; ++i) {
+    int len = dist_size(gen);
+    out[i].len = len;
+    out[i].ptr = buf;
+
+    bool do_prefix = dist_has_prefix(gen) && i > 0;
+    std::uniform_int_distribution<int> d4(min_size, len);

Review Comment:
   Removed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: ARROW-17619: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1404940303

   note: this should support BYTE_ARRAY and FIXED_LEN_BYTE_ARRAY. [PARQUET-2231](https://issues.apache.org/jira/browse/PARQUET-2231)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1146208117


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3062,226 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+    uint32_t previous_len = byte_width;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = byte_width;
+          } else {
+            uint32_t j = 0;
+            while (j < previous_len) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(byte_width - j);
+            last_value_ =

Review Comment:
   Got it! Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] wgtmac commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1147031566


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3064,269 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {

Review Comment:
   ```suggestion
       if (valid_bits != nullptr) {
   ```
   
   Uppercase `NULLPTR` is only used as default argument in the public header



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3064,269 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {

Review Comment:
   Should we convert FixedSizeBinaryArray to BinaryArray and then reuse code from PutBinaryArray? Or convert `FixedSizeBinaryArray` and `BinaryArray` to buffers with std::string_view type as an intermediate layer. It seems that `PutFixedLenByteArray` and `PutBinaryArray` share a lot of common logic.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1147030969


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3062,226 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+    uint32_t previous_len = byte_width;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = byte_width;
+          } else {
+            uint32_t j = 0;
+            while (j < previous_len) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(byte_width - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_TRUE(view.size() > kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = src.len;
+          } else {
+            uint32_t j = 0;
+            while (j < std::min(previous_len, src.len)) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  string_view last_value_;
+};
+
+template <>
+inline void DeltaByteArrayEncoder<FLBAType>::Put(const FixedLenByteArray* src,
+                                                 int num_values) {
+  if (descr_->type_length() == 0) {
+    return;
+  }
+  for (int i = 0; i < num_values; ++i) {
+    // Write the result to the output stream
+    DCHECK(src[i].ptr != nullptr) << "Value ptr cannot be NULL";
+    PARQUET_THROW_NOT_OK(sink_.Append(src[i].ptr, descr_->type_length()));
+  }
+}
+
+template <>
+void DeltaByteArrayEncoder<FLBAType>::Put(const ::arrow::Array& values) {
+  if (!::arrow::is_fixed_size_binary(values.type_id())) {
+    throw ParquetException("Only FixedSizeBinaryArray and subclasses supported");
+  }
+  PutFixedLenByteArray(checked_cast<const ::arrow::FixedSizeBinaryArray&>(values));
+}
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const ::arrow::Array& values) {
+  AssertBaseBinary(values);
+  if (::arrow::is_binary_like(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::BinaryArray&>(values));
+  } else if (::arrow::is_large_binary_like(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::LargeBinaryArray&>(values));
+  }
+}
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values);

Review Comment:
   Changed to:
   ```
   ArrowPoolVector<int32_t> prefix_lengths(num_values, ::arrow::stl::allocator<int32_t>(pool_));
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1147029451


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,11 +3062,226 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  void PutFixedLenByteArray(const ::arrow::FixedSizeBinaryArray& array) {
+    const uint32_t byte_width = array.byte_width();
+    uint32_t previous_len = byte_width;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<::arrow::FixedSizeBinaryType>(
+        *array.data(),
+        [&](::std::string_view view) {
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = byte_width;
+          } else {
+            uint32_t j = 0;
+            while (j < previous_len) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(byte_width - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    uint32_t previous_len = 0;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_TRUE(view.size() > kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          const ByteArray src{view};
+          if (ARROW_PREDICT_TRUE(last_value_.empty())) {
+            last_value_ = view;
+            suffix_encoder_.Put(&src, 1);
+            prefix_length_encoder_.Put({static_cast<int32_t>(0)}, 1);
+            previous_len = src.len;
+          } else {
+            uint32_t j = 0;
+            while (j < std::min(previous_len, src.len)) {
+              if (last_value_[j] != view[j]) {
+                break;
+              }
+              j++;
+            }
+            previous_len = j;
+            prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+            const uint8_t* suffix_ptr = src.ptr + j;
+            const uint32_t suffix_length = static_cast<uint32_t>(src.len - j);
+            last_value_ =
+                string_view{reinterpret_cast<const char*>(suffix_ptr), suffix_length};
+            const ByteArray suffix(suffix_length, suffix_ptr);
+            suffix_encoder_.Put(&suffix, 1);
+          }
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  string_view last_value_;
+};
+
+template <>
+inline void DeltaByteArrayEncoder<FLBAType>::Put(const FixedLenByteArray* src,
+                                                 int num_values) {
+  if (descr_->type_length() == 0) {
+    return;
+  }
+  for (int i = 0; i < num_values; ++i) {
+    // Write the result to the output stream
+    DCHECK(src[i].ptr != nullptr) << "Value ptr cannot be NULL";
+    PARQUET_THROW_NOT_OK(sink_.Append(src[i].ptr, descr_->type_length()));
+  }
+}
+
+template <>
+void DeltaByteArrayEncoder<FLBAType>::Put(const ::arrow::Array& values) {
+  if (!::arrow::is_fixed_size_binary(values.type_id())) {
+    throw ParquetException("Only FixedSizeBinaryArray and subclasses supported");
+  }
+  PutFixedLenByteArray(checked_cast<const ::arrow::FixedSizeBinaryArray&>(values));
+}
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const ::arrow::Array& values) {
+  AssertBaseBinary(values);
+  if (::arrow::is_binary_like(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::BinaryArray&>(values));
+  } else if (::arrow::is_large_binary_like(values.type_id())) {
+    PutBinaryArray(checked_cast<const ::arrow::LargeBinaryArray&>(values));
+  }
+}
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values);
+
+  if (ARROW_PREDICT_TRUE(last_value_.empty())) {

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1143978524


##########
cpp/src/parquet/encoding.cc:
##########
@@ -2900,8 +2929,222 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+// This is also known as incremental encoding or front compression: for each element in a
+// sequence of strings, store the prefix length of the previous entry plus the suffix.
+//
+// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+// followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+ public:
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr, MemoryPool* pool)
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY,
+                    pool = ::arrow::default_memory_pool()),

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] wgtmac commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1492219579

   Please do not forget to update `parquet.rst` for the new encoder implementation.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1157447585


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1275,6 +1280,37 @@ struct ArrowBinaryHelper {
   int64_t chunk_space_remaining;
 };
 
+template <>
+struct ArrowBinaryHelper<FLBAType> {
+  explicit ArrowBinaryHelper(EncodingTraits<FLBAType>::Accumulator* builder) {
+    this->builder = builder;
+    this->chunk_space_remaining =

Review Comment:
   Added a check.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1497045011

   Thanks for the review @wgtmac !


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1159713114


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3083,242 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, nullptr)) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          buffer_, ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);

Review Comment:
   `PutBinaryArray` now passes, but could perhaps be optimized:
   ```cpp
   if (suffix_length == 0) {
     const auto suffix_ptr = reinterpret_cast<const uint8_t*>("");
     const ByteArray suffix(suffix_length, suffix_ptr);
     suffix_encoder_.Put(&suffix, 1);
     return Status::OK();
   }
   ```
   
   Trying with:
   ```cpp
   const ByteArray suffix(suffix_length, nullptr);
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1162177502


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3083,242 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          buffer_, ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  std::unique_ptr<::arrow::Buffer> buffer_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw Status::Invalid("Put not implemented for " + this->descr_->ToString());

Review Comment:
   Changed.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3083,242 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(
+          buffer_, ::arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer_->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  std::unique_ptr<::arrow::Buffer> buffer_;
+  const ByteArray kEmpty;
+};
+
+template <typename DType>
+void DeltaByteArrayEncoder<DType>::Put(const T* src, int num_values) {
+  throw Status::Invalid("Put not implemented for " + this->descr_->ToString());
+}
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+  ArrowPoolVector<int32_t> prefix_lengths(num_values,
+                                          ::arrow::stl::allocator<int32_t>(pool_));
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i++) {
+    // Convert to ByteArray, so we can pass to the suffix_encoder_.
+    const ByteArray value = src[i];
+    if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+      throw Status::Invalid("Parquet cannot store strings with size 2GB or more");

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1503022127

   @pitrou do you mean you would prefer this is merged immediately after the 12.0.0 window so we have until 13.0.0 to observe for bugs? Or you mean we need another round of review?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1531992072

   LGTM now, I'm not a committer so no idea about how can this patch be merged ╮( ̄▽ ̄"")╭


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] wgtmac commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1257268192


##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1910,5 +1909,310 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void InitData(int nvalues, double null_probability) {
+    const int seed = 42;
+    auto rand = ::arrow::random::RandomArrayGenerator(seed);
+    const int min_prefix_length = 0;
+    const int max_prefix_length = 100;
+    const int max_element_length = 1000;
+    const double prefixed_probability = 0.5;

Review Comment:
   ```suggestion
       constexpr int kMinPrefixLength = 0;
       constexpr int kMaxPrefixLength = 100;
       constexpr int kMaxElementLength = 1000;
       constexpr double kPrefixedProbability = 0.5;
       constexpr int kSeed = 42;
       auto rand = ::arrow::random::RandomArrayGenerator(kSeed);
   ```
   
   Better to follow the convention.



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1910,5 +1909,310 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void InitData(int nvalues, double null_probability) {
+    const int seed = 42;
+    auto rand = ::arrow::random::RandomArrayGenerator(seed);
+    const int min_prefix_length = 0;
+    const int max_prefix_length = 100;
+    const int max_element_length = 1000;
+    const double prefixed_probability = 0.5;
+
+    const auto prefix_array = std::dynamic_pointer_cast<::arrow::StringArray>(
+        rand.String(nvalues, min_prefix_length, max_prefix_length, null_probability));
+    const auto do_prefix = std::dynamic_pointer_cast<::arrow::BooleanArray>(
+        rand.Boolean(nvalues,
+                     /*true_probability=*/prefixed_probability,
+                     /*null_probability=*/0.0));
+    ::arrow::StringBuilder builder(::arrow::default_memory_pool());
+
+    std::string prefix = "";
+    for (int i = 0; i < nvalues; i++) {
+      if (prefix_array->IsNull(i)) {
+        PARQUET_THROW_NOT_OK(builder.AppendNull());
+      } else {
+        const std::string element = prefix_array->GetString(i);
+        if (do_prefix->Value(i) && prefix.length() < max_element_length) {
+          prefix = prefix.append(element);
+        } else {
+          prefix = element;
+        }
+        PARQUET_THROW_NOT_OK(builder.Append(prefix));
+      }
+    }
+
+    std::shared_ptr<::arrow::StringArray> array;
+    ASSERT_OK(builder.Finish(&array));
+    num_values_ = static_cast<int>(array->length() - array->null_count());
+    draws_ = reinterpret_cast<c_type*>(array->value_data()->mutable_data());
+  }
+
+  void Execute(int nvalues, double null_probability) {
+    InitData(nvalues, null_probability);
+    CheckRoundtrip();
+  }
+
+  void ExecuteSpaced(int nvalues, int repeats, int64_t valid_bits_offset,

Review Comment:
   `int repeats` seems not used?



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1910,5 +1909,310 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void InitData(int nvalues, double null_probability) {
+    const int seed = 42;
+    auto rand = ::arrow::random::RandomArrayGenerator(seed);
+    const int min_prefix_length = 0;
+    const int max_prefix_length = 100;
+    const int max_element_length = 1000;
+    const double prefixed_probability = 0.5;
+
+    const auto prefix_array = std::dynamic_pointer_cast<::arrow::StringArray>(
+        rand.String(nvalues, min_prefix_length, max_prefix_length, null_probability));
+    const auto do_prefix = std::dynamic_pointer_cast<::arrow::BooleanArray>(
+        rand.Boolean(nvalues,
+                     /*true_probability=*/prefixed_probability,
+                     /*null_probability=*/0.0));
+    ::arrow::StringBuilder builder(::arrow::default_memory_pool());
+
+    std::string prefix = "";
+    for (int i = 0; i < nvalues; i++) {
+      if (prefix_array->IsNull(i)) {
+        PARQUET_THROW_NOT_OK(builder.AppendNull());
+      } else {
+        const std::string element = prefix_array->GetString(i);
+        if (do_prefix->Value(i) && prefix.length() < max_element_length) {
+          prefix = prefix.append(element);
+        } else {
+          prefix = element;
+        }
+        PARQUET_THROW_NOT_OK(builder.Append(prefix));
+      }
+    }
+
+    std::shared_ptr<::arrow::StringArray> array;
+    ASSERT_OK(builder.Finish(&array));
+    num_values_ = static_cast<int>(array->length() - array->null_count());
+    draws_ = reinterpret_cast<c_type*>(array->value_data()->mutable_data());
+  }
+
+  void Execute(int nvalues, double null_probability) {
+    InitData(nvalues, null_probability);
+    CheckRoundtrip();
+  }
+
+  void ExecuteSpaced(int nvalues, int repeats, int64_t valid_bits_offset,
+                     double null_probability) {
+    InitData(nvalues, null_probability);
+
+    int64_t size = num_values_ + valid_bits_offset;
+    auto rand = ::arrow::random::RandomArrayGenerator(1923);
+    const auto array = rand.UInt8(size, 0, 100, null_probability);
+    const auto valid_bits = array->null_bitmap_data();
+    if (valid_bits) {
+      CheckRoundtripSpaced(valid_bits, valid_bits_offset);
+    }
+  }
+
+  void CheckRoundtrip() override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+using TestDeltaByteArrayEncodingTypes = ::testing::Types<ByteArrayType, FLBAType>;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  // TODO
+
+  //  ASSERT_NO_FATAL_FAILURE(this->Execute(250, /*null_probability*/ 0));
+  //  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+  //      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_probability*/
+  //      0));
+  //
+  //  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, /*null_probability*/ 0.1));
+  //  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+  //      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+  //      /*null_probability*/ 0.5));
+}
+
+template <typename Type>
+class DeltaByteArrayEncodingDirectPut : public TestEncodingBase<Type> {
+ public:
+  std::unique_ptr<TypedEncoder<Type>> encoder =
+      MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+  std::unique_ptr<TypedDecoder<Type>> decoder =
+      MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+
+  void CheckDirectPut(std::shared_ptr<::arrow::Array> array) {
+    ASSERT_NO_THROW(encoder->Put(*array));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(array->length() - array->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<Type>::Accumulator acc;
+    if (::arrow::is_string(array->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(array->length()),
+                                   static_cast<int>(array->null_count()),
+                                   array->null_bitmap_data(), array->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(array->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    ::arrow::AssertArraysEqual(*array, *result);
+  }
+
+  void CheckRoundtrip() override {
+    const int64_t size = 50;
+    const int32_t min_length = 0;
+    const int32_t max_length = 10;
+    const int32_t num_unique = 10;
+    const double null_probability = 0.25;
+
+    ::arrow::random::RandomArrayGenerator rag{42};
+    std::shared_ptr<::arrow::Array> values =
+        rag.String(0, min_length, max_length, null_probability);
+    CheckDirectPut(values);
+
+    for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+      rag = ::arrow::random::RandomArrayGenerator(seed);
+      values = rag.String(size, min_length, max_length, null_probability);
+      CheckDirectPut(values);
+
+      values = rag.BinaryWithRepeats(size, num_unique, min_length, max_length,
+                                     null_probability);
+      CheckDirectPut(values);
+    }
+  }
+
+  void Execute() { CheckRoundtrip(); }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+using DeltaByteArrayEncodingDirectPutTypes =
+    ::testing::Types<ByteArrayType>;  // TODO: FLBAType
+TYPED_TEST_SUITE(DeltaByteArrayEncodingDirectPut, DeltaByteArrayEncodingDirectPutTypes);
+
+TYPED_TEST(DeltaByteArrayEncodingDirectPut, DirectPut) {
+  ASSERT_NO_FATAL_FAILURE(this->CheckRoundtrip());
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
+  auto CheckEncode = [](std::shared_ptr<::arrow::Array> values,
+                        std::shared_ptr<Buffer> encoded) {

Review Comment:
   Same for below.



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1910,5 +1909,310 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void InitData(int nvalues, double null_probability) {
+    const int seed = 42;
+    auto rand = ::arrow::random::RandomArrayGenerator(seed);
+    const int min_prefix_length = 0;
+    const int max_prefix_length = 100;
+    const int max_element_length = 1000;
+    const double prefixed_probability = 0.5;
+
+    const auto prefix_array = std::dynamic_pointer_cast<::arrow::StringArray>(
+        rand.String(nvalues, min_prefix_length, max_prefix_length, null_probability));
+    const auto do_prefix = std::dynamic_pointer_cast<::arrow::BooleanArray>(
+        rand.Boolean(nvalues,
+                     /*true_probability=*/prefixed_probability,
+                     /*null_probability=*/0.0));
+    ::arrow::StringBuilder builder(::arrow::default_memory_pool());
+
+    std::string prefix = "";
+    for (int i = 0; i < nvalues; i++) {
+      if (prefix_array->IsNull(i)) {
+        PARQUET_THROW_NOT_OK(builder.AppendNull());
+      } else {
+        const std::string element = prefix_array->GetString(i);
+        if (do_prefix->Value(i) && prefix.length() < max_element_length) {
+          prefix = prefix.append(element);
+        } else {
+          prefix = element;
+        }
+        PARQUET_THROW_NOT_OK(builder.Append(prefix));
+      }
+    }
+
+    std::shared_ptr<::arrow::StringArray> array;
+    ASSERT_OK(builder.Finish(&array));
+    num_values_ = static_cast<int>(array->length() - array->null_count());
+    draws_ = reinterpret_cast<c_type*>(array->value_data()->mutable_data());
+  }
+
+  void Execute(int nvalues, double null_probability) {
+    InitData(nvalues, null_probability);
+    CheckRoundtrip();
+  }
+
+  void ExecuteSpaced(int nvalues, int repeats, int64_t valid_bits_offset,
+                     double null_probability) {
+    InitData(nvalues, null_probability);
+
+    int64_t size = num_values_ + valid_bits_offset;
+    auto rand = ::arrow::random::RandomArrayGenerator(1923);
+    const auto array = rand.UInt8(size, 0, 100, null_probability);

Review Comment:
   ```suggestion
       const auto array = rand.UInt8(size, /*min=*/0, /*max=*/100, null_probability);
   ```



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1910,5 +1909,310 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void InitData(int nvalues, double null_probability) {
+    const int seed = 42;
+    auto rand = ::arrow::random::RandomArrayGenerator(seed);
+    const int min_prefix_length = 0;
+    const int max_prefix_length = 100;
+    const int max_element_length = 1000;
+    const double prefixed_probability = 0.5;
+
+    const auto prefix_array = std::dynamic_pointer_cast<::arrow::StringArray>(
+        rand.String(nvalues, min_prefix_length, max_prefix_length, null_probability));
+    const auto do_prefix = std::dynamic_pointer_cast<::arrow::BooleanArray>(
+        rand.Boolean(nvalues,
+                     /*true_probability=*/prefixed_probability,
+                     /*null_probability=*/0.0));
+    ::arrow::StringBuilder builder(::arrow::default_memory_pool());
+
+    std::string prefix = "";
+    for (int i = 0; i < nvalues; i++) {
+      if (prefix_array->IsNull(i)) {
+        PARQUET_THROW_NOT_OK(builder.AppendNull());
+      } else {
+        const std::string element = prefix_array->GetString(i);
+        if (do_prefix->Value(i) && prefix.length() < max_element_length) {
+          prefix = prefix.append(element);
+        } else {
+          prefix = element;
+        }
+        PARQUET_THROW_NOT_OK(builder.Append(prefix));
+      }
+    }
+
+    std::shared_ptr<::arrow::StringArray> array;
+    ASSERT_OK(builder.Finish(&array));
+    num_values_ = static_cast<int>(array->length() - array->null_count());
+    draws_ = reinterpret_cast<c_type*>(array->value_data()->mutable_data());
+  }
+
+  void Execute(int nvalues, double null_probability) {
+    InitData(nvalues, null_probability);
+    CheckRoundtrip();
+  }
+
+  void ExecuteSpaced(int nvalues, int repeats, int64_t valid_bits_offset,
+                     double null_probability) {
+    InitData(nvalues, null_probability);
+
+    int64_t size = num_values_ + valid_bits_offset;
+    auto rand = ::arrow::random::RandomArrayGenerator(1923);
+    const auto array = rand.UInt8(size, 0, 100, null_probability);
+    const auto valid_bits = array->null_bitmap_data();
+    if (valid_bits) {
+      CheckRoundtripSpaced(valid_bits, valid_bits_offset);
+    }
+  }
+
+  void CheckRoundtrip() override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+using TestDeltaByteArrayEncodingTypes = ::testing::Types<ByteArrayType, FLBAType>;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  // TODO
+
+  //  ASSERT_NO_FATAL_FAILURE(this->Execute(250, /*null_probability*/ 0));
+  //  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+  //      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_probability*/
+  //      0));
+  //
+  //  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, /*null_probability*/ 0.1));
+  //  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+  //      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+  //      /*null_probability*/ 0.5));
+}
+
+template <typename Type>
+class DeltaByteArrayEncodingDirectPut : public TestEncodingBase<Type> {
+ public:
+  std::unique_ptr<TypedEncoder<Type>> encoder =
+      MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+  std::unique_ptr<TypedDecoder<Type>> decoder =
+      MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+
+  void CheckDirectPut(std::shared_ptr<::arrow::Array> array) {
+    ASSERT_NO_THROW(encoder->Put(*array));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(array->length() - array->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<Type>::Accumulator acc;
+    if (::arrow::is_string(array->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(array->length()),
+                                   static_cast<int>(array->null_count()),
+                                   array->null_bitmap_data(), array->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(array->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    ::arrow::AssertArraysEqual(*array, *result);
+  }
+
+  void CheckRoundtrip() override {
+    const int64_t size = 50;
+    const int32_t min_length = 0;
+    const int32_t max_length = 10;
+    const int32_t num_unique = 10;
+    const double null_probability = 0.25;
+
+    ::arrow::random::RandomArrayGenerator rag{42};

Review Comment:
   ```suggestion
       constexpr int64_t kSize = 50;
       constexpr int32_t kMinLength = 0;
       constexpr int32_t kMaxLength = 10;
       constexpr int32_t kNumUnique = 10;
       constexpr double kNullProbability = 0.25;
       constexpr int kSeed = 42;
       ::arrow::random::RandomArrayGenerator rag{kSeed};
   ```



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1910,5 +1909,310 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void InitData(int nvalues, double null_probability) {
+    const int seed = 42;
+    auto rand = ::arrow::random::RandomArrayGenerator(seed);
+    const int min_prefix_length = 0;
+    const int max_prefix_length = 100;
+    const int max_element_length = 1000;
+    const double prefixed_probability = 0.5;
+
+    const auto prefix_array = std::dynamic_pointer_cast<::arrow::StringArray>(
+        rand.String(nvalues, min_prefix_length, max_prefix_length, null_probability));
+    const auto do_prefix = std::dynamic_pointer_cast<::arrow::BooleanArray>(
+        rand.Boolean(nvalues,
+                     /*true_probability=*/prefixed_probability,
+                     /*null_probability=*/0.0));
+    ::arrow::StringBuilder builder(::arrow::default_memory_pool());
+
+    std::string prefix = "";
+    for (int i = 0; i < nvalues; i++) {
+      if (prefix_array->IsNull(i)) {
+        PARQUET_THROW_NOT_OK(builder.AppendNull());
+      } else {
+        const std::string element = prefix_array->GetString(i);
+        if (do_prefix->Value(i) && prefix.length() < max_element_length) {
+          prefix = prefix.append(element);
+        } else {
+          prefix = element;
+        }
+        PARQUET_THROW_NOT_OK(builder.Append(prefix));
+      }
+    }
+
+    std::shared_ptr<::arrow::StringArray> array;
+    ASSERT_OK(builder.Finish(&array));
+    num_values_ = static_cast<int>(array->length() - array->null_count());
+    draws_ = reinterpret_cast<c_type*>(array->value_data()->mutable_data());
+  }
+
+  void Execute(int nvalues, double null_probability) {
+    InitData(nvalues, null_probability);
+    CheckRoundtrip();
+  }
+
+  void ExecuteSpaced(int nvalues, int repeats, int64_t valid_bits_offset,
+                     double null_probability) {
+    InitData(nvalues, null_probability);
+
+    int64_t size = num_values_ + valid_bits_offset;
+    auto rand = ::arrow::random::RandomArrayGenerator(1923);
+    const auto array = rand.UInt8(size, 0, 100, null_probability);
+    const auto valid_bits = array->null_bitmap_data();
+    if (valid_bits) {
+      CheckRoundtripSpaced(valid_bits, valid_bits_offset);
+    }
+  }
+
+  void CheckRoundtrip() override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+using TestDeltaByteArrayEncodingTypes = ::testing::Types<ByteArrayType, FLBAType>;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  // TODO
+
+  //  ASSERT_NO_FATAL_FAILURE(this->Execute(250, /*null_probability*/ 0));
+  //  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+  //      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_probability*/
+  //      0));
+  //
+  //  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, /*null_probability*/ 0.1));
+  //  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+  //      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+  //      /*null_probability*/ 0.5));

Review Comment:
   Could we uncomment these lines now?



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1910,5 +1909,310 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void InitData(int nvalues, double null_probability) {
+    const int seed = 42;
+    auto rand = ::arrow::random::RandomArrayGenerator(seed);
+    const int min_prefix_length = 0;
+    const int max_prefix_length = 100;
+    const int max_element_length = 1000;
+    const double prefixed_probability = 0.5;
+
+    const auto prefix_array = std::dynamic_pointer_cast<::arrow::StringArray>(
+        rand.String(nvalues, min_prefix_length, max_prefix_length, null_probability));
+    const auto do_prefix = std::dynamic_pointer_cast<::arrow::BooleanArray>(
+        rand.Boolean(nvalues,
+                     /*true_probability=*/prefixed_probability,
+                     /*null_probability=*/0.0));
+    ::arrow::StringBuilder builder(::arrow::default_memory_pool());
+
+    std::string prefix = "";
+    for (int i = 0; i < nvalues; i++) {
+      if (prefix_array->IsNull(i)) {
+        PARQUET_THROW_NOT_OK(builder.AppendNull());
+      } else {
+        const std::string element = prefix_array->GetString(i);
+        if (do_prefix->Value(i) && prefix.length() < max_element_length) {
+          prefix = prefix.append(element);
+        } else {
+          prefix = element;
+        }
+        PARQUET_THROW_NOT_OK(builder.Append(prefix));
+      }
+    }
+
+    std::shared_ptr<::arrow::StringArray> array;
+    ASSERT_OK(builder.Finish(&array));
+    num_values_ = static_cast<int>(array->length() - array->null_count());
+    draws_ = reinterpret_cast<c_type*>(array->value_data()->mutable_data());
+  }
+
+  void Execute(int nvalues, double null_probability) {
+    InitData(nvalues, null_probability);
+    CheckRoundtrip();
+  }
+
+  void ExecuteSpaced(int nvalues, int repeats, int64_t valid_bits_offset,
+                     double null_probability) {
+    InitData(nvalues, null_probability);
+
+    int64_t size = num_values_ + valid_bits_offset;
+    auto rand = ::arrow::random::RandomArrayGenerator(1923);
+    const auto array = rand.UInt8(size, 0, 100, null_probability);
+    const auto valid_bits = array->null_bitmap_data();
+    if (valid_bits) {
+      CheckRoundtripSpaced(valid_bits, valid_bits_offset);
+    }
+  }
+
+  void CheckRoundtrip() override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+using TestDeltaByteArrayEncodingTypes = ::testing::Types<ByteArrayType, FLBAType>;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  // TODO
+
+  //  ASSERT_NO_FATAL_FAILURE(this->Execute(250, /*null_probability*/ 0));
+  //  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+  //      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_probability*/
+  //      0));
+  //
+  //  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, /*null_probability*/ 0.1));
+  //  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+  //      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+  //      /*null_probability*/ 0.5));
+}
+
+template <typename Type>
+class DeltaByteArrayEncodingDirectPut : public TestEncodingBase<Type> {
+ public:
+  std::unique_ptr<TypedEncoder<Type>> encoder =
+      MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+  std::unique_ptr<TypedDecoder<Type>> decoder =
+      MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+
+  void CheckDirectPut(std::shared_ptr<::arrow::Array> array) {
+    ASSERT_NO_THROW(encoder->Put(*array));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(array->length() - array->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<Type>::Accumulator acc;
+    if (::arrow::is_string(array->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(array->length()),
+                                   static_cast<int>(array->null_count()),
+                                   array->null_bitmap_data(), array->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(array->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    ::arrow::AssertArraysEqual(*array, *result);
+  }
+
+  void CheckRoundtrip() override {
+    const int64_t size = 50;
+    const int32_t min_length = 0;
+    const int32_t max_length = 10;
+    const int32_t num_unique = 10;
+    const double null_probability = 0.25;
+
+    ::arrow::random::RandomArrayGenerator rag{42};
+    std::shared_ptr<::arrow::Array> values =
+        rag.String(0, min_length, max_length, null_probability);
+    CheckDirectPut(values);
+
+    for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+      rag = ::arrow::random::RandomArrayGenerator(seed);
+      values = rag.String(size, min_length, max_length, null_probability);
+      CheckDirectPut(values);
+
+      values = rag.BinaryWithRepeats(size, num_unique, min_length, max_length,
+                                     null_probability);
+      CheckDirectPut(values);
+    }
+  }
+
+  void Execute() { CheckRoundtrip(); }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+using DeltaByteArrayEncodingDirectPutTypes =
+    ::testing::Types<ByteArrayType>;  // TODO: FLBAType
+TYPED_TEST_SUITE(DeltaByteArrayEncodingDirectPut, DeltaByteArrayEncodingDirectPutTypes);
+
+TYPED_TEST(DeltaByteArrayEncodingDirectPut, DirectPut) {
+  ASSERT_NO_FATAL_FAILURE(this->CheckRoundtrip());
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
+  auto CheckEncode = [](std::shared_ptr<::arrow::Array> values,
+                        std::shared_ptr<Buffer> encoded) {
+    auto encoder = MakeTypedEncoder<ByteArrayType>(Encoding::DELTA_BYTE_ARRAY);
+    ASSERT_NO_THROW(encoder->Put(*values));
+    auto buf = encoder->FlushValues();
+    ASSERT_TRUE(encoded->Equals(*buf));
+  };
+
+  auto arrayToI32 = [](const std::shared_ptr<::arrow::Array>& lengths) {

Review Comment:
   ```suggestion
     auto ArrayToInt32Vector = [](const std::shared_ptr<::arrow::Array>& lengths) {
   ```



##########
cpp/src/parquet/encoding_test.cc:
##########
@@ -1910,5 +1909,310 @@ TEST(DeltaLengthByteArrayEncodingAdHoc, ArrowDirectPut) {
   CheckDecode(encoded, ::arrow::ArrayFromJSON(::arrow::large_binary(), values));
 }
 
+// ----------------------------------------------------------------------
+// DELTA_BYTE_ARRAY encode/decode tests.
+
+template <typename Type>
+class TestDeltaByteArrayEncoding : public TestEncodingBase<Type> {
+ public:
+  using c_type = typename Type::c_type;
+  static constexpr int TYPE = Type::type_num;
+
+  void InitData(int nvalues, double null_probability) {
+    const int seed = 42;
+    auto rand = ::arrow::random::RandomArrayGenerator(seed);
+    const int min_prefix_length = 0;
+    const int max_prefix_length = 100;
+    const int max_element_length = 1000;
+    const double prefixed_probability = 0.5;
+
+    const auto prefix_array = std::dynamic_pointer_cast<::arrow::StringArray>(
+        rand.String(nvalues, min_prefix_length, max_prefix_length, null_probability));
+    const auto do_prefix = std::dynamic_pointer_cast<::arrow::BooleanArray>(
+        rand.Boolean(nvalues,
+                     /*true_probability=*/prefixed_probability,
+                     /*null_probability=*/0.0));
+    ::arrow::StringBuilder builder(::arrow::default_memory_pool());
+
+    std::string prefix = "";
+    for (int i = 0; i < nvalues; i++) {
+      if (prefix_array->IsNull(i)) {
+        PARQUET_THROW_NOT_OK(builder.AppendNull());
+      } else {
+        const std::string element = prefix_array->GetString(i);
+        if (do_prefix->Value(i) && prefix.length() < max_element_length) {
+          prefix = prefix.append(element);
+        } else {
+          prefix = element;
+        }
+        PARQUET_THROW_NOT_OK(builder.Append(prefix));
+      }
+    }
+
+    std::shared_ptr<::arrow::StringArray> array;
+    ASSERT_OK(builder.Finish(&array));
+    num_values_ = static_cast<int>(array->length() - array->null_count());
+    draws_ = reinterpret_cast<c_type*>(array->value_data()->mutable_data());
+  }
+
+  void Execute(int nvalues, double null_probability) {
+    InitData(nvalues, null_probability);
+    CheckRoundtrip();
+  }
+
+  void ExecuteSpaced(int nvalues, int repeats, int64_t valid_bits_offset,
+                     double null_probability) {
+    InitData(nvalues, null_probability);
+
+    int64_t size = num_values_ + valid_bits_offset;
+    auto rand = ::arrow::random::RandomArrayGenerator(1923);
+    const auto array = rand.UInt8(size, 0, 100, null_probability);
+    const auto valid_bits = array->null_bitmap_data();
+    if (valid_bits) {
+      CheckRoundtripSpaced(valid_bits, valid_bits_offset);
+    }
+  }
+
+  void CheckRoundtrip() override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+
+    encoder->Put(draws_, num_values_);
+    encode_buffer_ = encoder->FlushValues();
+
+    decoder->SetData(num_values_, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    int values_decoded = decoder->Decode(decode_buf_, num_values_);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<c_type>(decode_buf_, draws_, num_values_));
+  }
+
+  void CheckRoundtripSpaced(const uint8_t* valid_bits,
+                            int64_t valid_bits_offset) override {
+    auto encoder = MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY,
+                                          /*use_dictionary=*/false, descr_.get());
+    auto decoder = MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY, descr_.get());
+    int null_count = 0;
+    for (auto i = 0; i < num_values_; i++) {
+      if (!bit_util::GetBit(valid_bits, valid_bits_offset + i)) {
+        null_count++;
+      }
+    }
+
+    encoder->PutSpaced(draws_, num_values_, valid_bits, valid_bits_offset);
+    encode_buffer_ = encoder->FlushValues();
+    decoder->SetData(num_values_ - null_count, encode_buffer_->data(),
+                     static_cast<int>(encode_buffer_->size()));
+    auto values_decoded = decoder->DecodeSpaced(decode_buf_, num_values_, null_count,
+                                                valid_bits, valid_bits_offset);
+    ASSERT_EQ(num_values_, values_decoded);
+    ASSERT_NO_FATAL_FAILURE(VerifyResultsSpaced<c_type>(decode_buf_, draws_, num_values_,
+                                                        valid_bits, valid_bits_offset));
+  }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+using TestDeltaByteArrayEncodingTypes = ::testing::Types<ByteArrayType, FLBAType>;
+TYPED_TEST_SUITE(TestDeltaByteArrayEncoding, TestDeltaByteArrayEncodingTypes);
+
+TYPED_TEST(TestDeltaByteArrayEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(0, 0));
+  // TODO
+
+  //  ASSERT_NO_FATAL_FAILURE(this->Execute(250, /*null_probability*/ 0));
+  //  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+  //      /*nvalues*/ 1234, /*repeats*/ 1, /*valid_bits_offset*/ 64, /*null_probability*/
+  //      0));
+  //
+  //  ASSERT_NO_FATAL_FAILURE(this->Execute(2000, /*null_probability*/ 0.1));
+  //  ASSERT_NO_FATAL_FAILURE(this->ExecuteSpaced(
+  //      /*nvalues*/ 1234, /*repeats*/ 10, /*valid_bits_offset*/ 64,
+  //      /*null_probability*/ 0.5));
+}
+
+template <typename Type>
+class DeltaByteArrayEncodingDirectPut : public TestEncodingBase<Type> {
+ public:
+  std::unique_ptr<TypedEncoder<Type>> encoder =
+      MakeTypedEncoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+  std::unique_ptr<TypedDecoder<Type>> decoder =
+      MakeTypedDecoder<Type>(Encoding::DELTA_BYTE_ARRAY);
+
+  void CheckDirectPut(std::shared_ptr<::arrow::Array> array) {
+    ASSERT_NO_THROW(encoder->Put(*array));
+    auto buf = encoder->FlushValues();
+
+    int num_values = static_cast<int>(array->length() - array->null_count());
+    decoder->SetData(num_values, buf->data(), static_cast<int>(buf->size()));
+
+    typename EncodingTraits<Type>::Accumulator acc;
+    if (::arrow::is_string(array->type()->id())) {
+      acc.builder = std::make_unique<::arrow::StringBuilder>();
+    } else {
+      acc.builder = std::make_unique<::arrow::BinaryBuilder>();
+    }
+
+    ASSERT_EQ(num_values,
+              decoder->DecodeArrow(static_cast<int>(array->length()),
+                                   static_cast<int>(array->null_count()),
+                                   array->null_bitmap_data(), array->offset(), &acc));
+
+    std::shared_ptr<::arrow::Array> result;
+    ASSERT_OK(acc.builder->Finish(&result));
+    ASSERT_EQ(array->length(), result->length());
+    ASSERT_OK(result->ValidateFull());
+
+    ::arrow::AssertArraysEqual(*array, *result);
+  }
+
+  void CheckRoundtrip() override {
+    const int64_t size = 50;
+    const int32_t min_length = 0;
+    const int32_t max_length = 10;
+    const int32_t num_unique = 10;
+    const double null_probability = 0.25;
+
+    ::arrow::random::RandomArrayGenerator rag{42};
+    std::shared_ptr<::arrow::Array> values =
+        rag.String(0, min_length, max_length, null_probability);
+    CheckDirectPut(values);
+
+    for (auto seed : {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) {
+      rag = ::arrow::random::RandomArrayGenerator(seed);
+      values = rag.String(size, min_length, max_length, null_probability);
+      CheckDirectPut(values);
+
+      values = rag.BinaryWithRepeats(size, num_unique, min_length, max_length,
+                                     null_probability);
+      CheckDirectPut(values);
+    }
+  }
+
+  void Execute() { CheckRoundtrip(); }
+
+ protected:
+  USING_BASE_MEMBERS();
+};
+
+using DeltaByteArrayEncodingDirectPutTypes =
+    ::testing::Types<ByteArrayType>;  // TODO: FLBAType
+TYPED_TEST_SUITE(DeltaByteArrayEncodingDirectPut, DeltaByteArrayEncodingDirectPutTypes);
+
+TYPED_TEST(DeltaByteArrayEncodingDirectPut, DirectPut) {
+  ASSERT_NO_FATAL_FAILURE(this->CheckRoundtrip());
+}
+
+TEST(DeltaByteArrayEncodingAdHoc, ArrowDirectPut) {
+  auto CheckEncode = [](std::shared_ptr<::arrow::Array> values,
+                        std::shared_ptr<Buffer> encoded) {

Review Comment:
   ```suggestion
     auto CheckEncode = [](const std::shared_ptr<::arrow::Array>& values,
                           const std::shared_ptr<Buffer>& encoded) {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] wgtmac commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1256835289


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,14 +3058,237 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
+  static constexpr std::string_view kEmpty = "";
+
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(descr, pool),

Review Comment:
   Sorry for confusion, I think we should only pass `descr` to suffix_encoder_ as it applies to ByteArrayType. But prefix_length_encoder_ is only for integer types, passing descr to it may introduce weird behavior. What about keep passing nullptr for now? Though internally the DeltaBitPackEncoder does not use the type, I am just afraid it will break in the future. WDYT?
   
   cc @mapleFU @pitrou 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1627390933

   Could this patch be review? I'll take a round tomorrow!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] wgtmac commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1256043091


##########
cpp/src/parquet/encoding.cc:
##########
@@ -2842,7 +2888,7 @@ class DeltaLengthByteArrayDecoder : public DecoderImpl,
 
   std::shared_ptr<::arrow::bit_util::BitReader> decoder_;
   DeltaBitPackDecoder<Int32Type> len_decoder_;
-  int num_valid_values_;
+  int num_valid_values_{0};

Review Comment:
   This is initialized via `SetData`. But it doesn't hurt to provide a default value here. Would you want to do the same thing for `length_idx_`?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,25 +1240,35 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
+template <typename DType>
+struct ArrowBinaryHelper;

Review Comment:
   Just a reminder that this may conflict with https://github.com/apache/arrow/pull/35825/files once it gets merged.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,230 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),

Review Comment:
   Why passing nullptr instead of descr? Is it safe to do so?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,230 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        auto view = visitor[i + j];
+        len = visitor.len(i + j);
+
+        uint32_t k = 0;
+        const uint32_t common_length =
+            std::min(len, static_cast<uint32_t>(last_value_view.length()));
+        while (k < common_length) {
+          if (last_value_view[k] != view[k]) {
+            break;
+          }
+          k++;
+        }
+
+        last_value_view = view;
+        prefix_lengths[j] = k;
+        const uint32_t suffix_length = len - k;
+        const uint8_t* suffix_ptr = src[i + j].ptr + k;
+
+        // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+        const ByteArray suffix(suffix_length, suffix_ptr);
+        suffixes[j] = suffix;
+      }
+      suffix_encoder_.Put(suffixes.data(), batch_size);
+      prefix_length_encoder_.Put(prefix_lengths.data(), batch_size);
+    }
+    last_value_ = last_value_view;
+  }
+
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.length());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t len = src.len;
+          const uint32_t common_length = std::min(previous_len, len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+};
+
+struct ByteArrayVisitor {
+  const ByteArray* src;
+  const uint32_t type_length;

Review Comment:
   Add a comment to say type_length is not used for ByteArratType or simply add `[[maybe_unused]]`?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,230 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();
+
+    std::string_view last_value_view = last_value_;
+    constexpr int kBatchSize = 256;
+    std::array<int32_t, kBatchSize> prefix_lengths;
+    std::array<ByteArray, kBatchSize> suffixes;
+    auto visitor = VisitorType{src, len};
+
+    for (int i = 0; i < num_values; i += kBatchSize) {
+      const int batch_size = std::min(kBatchSize, num_values - i);
+
+      for (int j = 0; j < batch_size; ++j) {
+        auto view = visitor[i + j];

Review Comment:
   `i + j` is computed for three times, probably worth a local variable?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3188,18 +3447,54 @@ class DeltaByteArrayDecoder : public DecoderImpl,
     return Status::OK();
   }
 
+  MemoryPool* pool_;
+
+ private:
   std::shared_ptr<::arrow::bit_util::BitReader> decoder_;
   DeltaBitPackDecoder<Int32Type> prefix_len_decoder_;
   DeltaLengthByteArrayDecoder suffix_decoder_;
   std::string last_value_;
   // string buffer for last value in previous page
   std::string last_value_in_previous_page_;
-  int num_valid_values_;
+  int num_valid_values_{0};

Review Comment:
   ditto, is this necessary?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,230 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>

Review Comment:
   ```suggestion
   // DeltaByteArrayEncoder
   
   constexpr std::string_view kEmpty = "";
   
   template <typename DType>
   ```



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,230 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}

Review Comment:
   I am afraid this may be dangerous. What about declaring the constant kEmpty outside like above, and then use it here:
   ```cpp
           empty_(kEmpty.size(), reinterpret_cast<const uint8_t*>(kEmpty.data())) {}
   ```



##########
cpp/src/parquet/encoding.cc:
##########
@@ -1238,25 +1240,35 @@ int PlainBooleanDecoder::Decode(bool* buffer, int max_values) {
   return max_values;
 }
 
-struct ArrowBinaryHelper {
+template <typename DType>
+struct ArrowBinaryHelper;
+
+template <>
+struct ArrowBinaryHelper<ByteArrayType> {
   explicit ArrowBinaryHelper(typename EncodingTraits<ByteArrayType>::Accumulator* out) {
     this->out = out;
     this->builder = out->builder.get();
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 this->builder->value_data_length(),
+                                                 &this->chunk_space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<ByteArrayType>");
+    }
     this->chunk_space_remaining =

Review Comment:
   This line would be redundant, it has been set via line 1251 already.



##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,230 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename VisitorType>
+  void PutInternal(const T* src, int num_values) {
+    if (num_values == 0) {
+      return;
+    }
+    uint32_t len = descr_->type_length();

Review Comment:
   IIRC, `descr_->type_length()` will be 0 for ByteArrayType, so it is only used for FLBAType, right?



##########
cpp/src/parquet/encoding.cc:
##########
@@ -1275,6 +1287,40 @@ struct ArrowBinaryHelper {
   int64_t chunk_space_remaining;
 };
 
+template <>
+struct ArrowBinaryHelper<FLBAType> {
+  explicit ArrowBinaryHelper(EncodingTraits<FLBAType>::Accumulator* builder) {
+    this->builder = builder;
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(::arrow::kBinaryMemoryLimit,
+                                                 this->builder->value_data_length(),
+                                                 &this->chunk_space_remaining))) {
+      throw ParquetException("excess expansion in ArrowBinaryHelper<FLBAType>");
+    }
+  }
+
+  Status PushChunk() {
+    std::shared_ptr<::arrow::Array> result;
+    RETURN_NOT_OK(builder->Finish(&result));
+    chunks.push_back(std::move(result));
+    chunk_space_remaining = ::arrow::kBinaryMemoryLimit;
+    return Status::OK();
+  }
+
+  bool CanFit(int64_t length) const { return length <= chunk_space_remaining; }
+
+  Status Append(const uint8_t* data, int32_t length) {
+    DCHECK(CanFit(length));
+    chunk_space_remaining -= length;
+    return builder->Append(data);
+  }
+
+  Status AppendNull() { return builder->AppendNull(); }
+
+  ::arrow::FixedSizeBinaryBuilder* builder;
+  std::vector<std::shared_ptr<::arrow::Array>> chunks;

Review Comment:
   I'm not sure if this has been discussed already. Would it help to change `struct EncodingTraits<FLBAType>` to be similar with `struct EncodingTraits<ByteArrayType>`? Something like:
   ```cpp
   template <>
   struct EncodingTraits<FLBAType> {
     using Encoder = FLBAEncoder;
     using Decoder = FLBADecoder;
   
     struct Accumulator {
       std::unique_ptr<::arrow::FixedSizeBinaryBuilder> builder;
       std::vector<std::shared_ptr<::arrow::Array>> chunks;
     };
   
     using ArrowType = ::arrow::FixedSizeBinaryType;
     using DictAccumulator = ::arrow::Dictionary32Builder<::arrow::FixedSizeBinaryType>;
   };
   ```
   
   This may help eliminate the duplication introduced by template specialization. You may simply define this:
   ```cpp
   template <typename DType, typename Enable = void>
   struct ArrowBinaryHelper;
   
   template <typename DType>
   struct ArrowBinaryHelper<DType, std::enable_if_t<std::is_same_v<DType, ByteArrayType> || std::is_same_v<DType, FLBAType>, void>)> {
     ...
   };
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1256732232


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,12 +3083,230 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),

Review Comment:
   I'm not sure about safety. Changed to use descr wherever possible.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1295883850


##########
cpp/src/parquet/test_util.h:
##########
@@ -783,8 +789,33 @@ inline void GenerateData<ByteArray>(int num_values, ByteArray* out,
   random_byte_array(num_values, 0, heap->data(), out, 2, max_byte_array_len);
 }
 
+template <typename T>

Review Comment:
   Perhaps add a comment explaining what this function does? For example:
   ```suggestion
   // Generate ByteArray or FLBA data where there is a given probability
   // for each value to share a common prefix with its predecessor.
   // This is useful to exercise prefix-based encodings such as DELTA_BYTE_ARRAY.
   template <typename T>
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1198516032


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3200,6 +3461,42 @@ class DeltaByteArrayDecoder : public DecoderImpl,
   std::shared_ptr<ResizableBuffer> buffered_data_;
 };
 
+class DeltaByteArrayDecoder : public DeltaByteArrayDecoderImpl<ByteArrayType> {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<ByteArrayType>;
+  using Base::DeltaByteArrayDecoderImpl;
+
+  int Decode(ByteArray* buffer, int max_values) override {
+    return GetInternal(buffer, max_values);
+  }
+};
+
+class DeltaByteArrayFLBADecoder : public DeltaByteArrayDecoderImpl<FLBAType>,
+                                  virtual public FLBADecoder {
+ public:
+  using Base = DeltaByteArrayDecoderImpl<FLBAType>;
+  using Base::DeltaByteArrayDecoderImpl;
+  using Base::pool_;
+
+  int Decode(FixedLenByteArray* buffer, int max_values) override {

Review Comment:
   I guess pitrou's idea is that:
   
   ```c++
     int Decode(FixedLenByteArray* buffer, int max_values) override {
       int decoded_values_size = max_values;
       // GetInternal currently only support ByteArray.
       std::vector<ByteArray> decode_byte_array(decoded_values_size);
       max_values = GetInternal(decode_byte_array.data(), max_values);
       for (int i = 0; i < max_values; i++) {
         if (ARROW_PREDICT_FALSE(decode_byte_array[i].len != descr_->type_length())) {
           throw ParquetException("Fixed length byte array length mismatch");
         }
         buffer[i].ptr = decode_byte_array[i].ptr + i * descr_->type_length();
       }
       return max_values;
     }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1542590479

   @mapleFU Yes, I'll do. Sorry!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on PR #14341:
URL: https://github.com/apache/arrow/pull/14341#issuecomment-1644128780

   @pitrou I think I addressed all but https://github.com/apache/arrow/pull/14341#discussion_r1268173036.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rok commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "rok (via GitHub)" <gi...@apache.org>.
rok commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1286005515


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3037,11 +3073,179 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_("") {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),

Review Comment:
   Duplicate: https://github.com/apache/arrow/pull/14341#discussion_r1161834118



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer [arrow]

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1386762899


##########
cpp/src/parquet/encoding.cc:
##########
@@ -1335,13 +1420,7 @@ class PlainByteArrayDecoder : public PlainDecoder<ByteArrayType>,
           if (ARROW_PREDICT_FALSE(len_ < increment)) {
             ParquetException::EofException();
           }
-          if (ARROW_PREDICT_FALSE(!helper.CanFit(value_len))) {
-            // This element would exceed the capacity of a chunk
-            RETURN_NOT_OK(helper.PushChunk());
-            RETURN_NOT_OK(helper.builder->Reserve(num_values - i));
-            RETURN_NOT_OK(helper.builder->ReserveData(
-                std::min<int64_t>(len_, helper.chunk_space_remaining)));

Review Comment:
   Why previously here `ReserveData` for `min(len_, helper.chunk_space_remaining)` here, wouldn't `len_` be too large @pitrou 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #14341: GH-32863: [C++][Parquet] Add DELTA_BYTE_ARRAY encoder to Parquet writer

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #14341:
URL: https://github.com/apache/arrow/pull/14341#discussion_r1197717260


##########
cpp/src/parquet/encoding.cc:
##########
@@ -3033,12 +3079,243 @@ class RleBooleanDecoder : public DecoderImpl, virtual public BooleanDecoder {
 // ----------------------------------------------------------------------
 // DELTA_BYTE_ARRAY
 
-class DeltaByteArrayDecoder : public DecoderImpl,
-                              virtual public TypedDecoder<ByteArrayType> {
+/// Delta Byte Array encoding also known as incremental encoding or front compression:
+/// for each element in a sequence of strings, store the prefix length of the previous
+/// entry plus the suffix.
+///
+/// This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED),
+/// followed by the suffixes encoded as delta length byte arrays
+/// (DELTA_LENGTH_BYTE_ARRAY).
+
+// ----------------------------------------------------------------------
+// DeltaByteArrayEncoder
+
+template <typename DType>
+class DeltaByteArrayEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
  public:
-  explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
+  using T = typename DType::c_type;
+
+  explicit DeltaByteArrayEncoder(const ColumnDescriptor* descr,
                                  MemoryPool* pool = ::arrow::default_memory_pool())
+      : EncoderImpl(descr, Encoding::DELTA_BYTE_ARRAY, pool),
+        sink_(pool),
+        prefix_length_encoder_(nullptr, pool),
+        suffix_encoder_(nullptr, pool),
+        last_value_(""),
+        kEmpty(ByteArray(0, reinterpret_cast<const uint8_t*>(""))),
+        prefix_lengths_(kBatchSize_, ::arrow::stl::allocator<int32_t>(pool_)) {}
+
+  std::shared_ptr<Buffer> FlushValues() override;
+
+  int64_t EstimatedDataEncodedSize() override {
+    return prefix_length_encoder_.EstimatedDataEncodedSize() +
+           suffix_encoder_.EstimatedDataEncodedSize();
+  }
+
+  using TypedEncoder<DType>::Put;
+
+  void Put(const ::arrow::Array& values) override;
+
+  void Put(const T* buffer, int num_values) override;
+
+  void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
+                 int64_t valid_bits_offset) override {
+    if (valid_bits != NULLPTR) {
+      PARQUET_ASSIGN_OR_THROW(auto buffer, ::arrow::AllocateBuffer(num_values * sizeof(T),
+                                                                   this->memory_pool()));
+      T* data = reinterpret_cast<T*>(buffer->mutable_data());
+      int num_valid_values = ::arrow::util::internal::SpacedCompress<T>(
+          src, num_values, valid_bits, valid_bits_offset, data);
+      Put(data, num_valid_values);
+    } else {
+      Put(src, num_values);
+    }
+  }
+
+ protected:
+  template <typename ArrayType>
+  void PutBinaryArray(const ArrayType& array) {
+    auto previous_len = static_cast<uint32_t>(last_value_.size());
+    std::string_view last_value_view = last_value_;
+
+    PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename ArrayType::TypeClass>(
+        *array.data(),
+        [&](::std::string_view view) {
+          if (ARROW_PREDICT_FALSE(view.size() >= kMaxByteArraySize)) {
+            return Status::Invalid("Parquet cannot store strings with size 2GB or more");
+          }
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray src{view};
+
+          uint32_t j = 0;
+          const uint32_t common_length = std::min(previous_len, src.len);
+          while (j < common_length) {
+            if (last_value_view[j] != view[j]) {
+              break;
+            }
+            j++;
+          }
+          previous_len = src.len;
+          prefix_length_encoder_.Put({static_cast<int32_t>(j)}, 1);
+
+          last_value_view = view;
+          const auto suffix_length = static_cast<uint32_t>(src.len - j);
+          if (suffix_length == 0) {
+            suffix_encoder_.Put(&kEmpty, 1);
+            return Status::OK();
+          }
+          const uint8_t* suffix_ptr = src.ptr + j;
+          // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+          const ByteArray suffix(suffix_length, suffix_ptr);
+          suffix_encoder_.Put(&suffix, 1);
+
+          return Status::OK();
+        },
+        []() { return Status::OK(); }));
+    last_value_ = last_value_view;
+  }
+
+  ::arrow::BufferBuilder sink_;
+  DeltaBitPackEncoder<Int32Type> prefix_length_encoder_;
+  DeltaLengthByteArrayEncoder<ByteArrayType> suffix_encoder_;
+  std::string last_value_;
+  const ByteArray kEmpty;
+  ArrowPoolVector<int32_t> prefix_lengths_;
+  static constexpr int kBatchSize_ = 256;
+};
+
+template <>
+void DeltaByteArrayEncoder<ByteArrayType>::Put(const ByteArray* src, int num_values) {
+  if (num_values == 0) {
+    return;
+  }
+
+  std::string_view last_value_view = last_value_;
+
+  for (int i = 0; i < num_values; i += kBatchSize_) {
+    const int batch_size = std::min(kBatchSize_, num_values - i);
+
+    for (int j = 0; j < batch_size; ++j) {
+      // Convert to ByteArray, so we can pass to the suffix_encoder_.
+      const ByteArray value = src[i + j];
+      if (ARROW_PREDICT_FALSE(value.len >= static_cast<int32_t>(kMaxByteArraySize))) {
+        throw ParquetException("Parquet cannot store strings with size 2GB or more");
+      }
+      auto view = std::string_view{value};
+
+      uint32_t k = 0;
+      const uint32_t common_length =
+          std::min(value.len, static_cast<uint32_t>(last_value_view.length()));
+      while (k < common_length) {
+        if (last_value_view[k] != view[k]) {
+          break;
+        }
+        k++;
+      }
+
+      last_value_view = view;
+      prefix_lengths_[j] = k;
+      const auto suffix_length = static_cast<uint32_t>(value.len - k);
+
+      if (suffix_length == 0) {
+        continue;
+      }
+      const uint8_t* suffix_ptr = value.ptr + k;
+      // Convert to ByteArray, so it can be passed to the suffix_encoder_.
+      const ByteArray suffix(suffix_length, suffix_ptr);
+      suffix_encoder_.Put(&suffix, 1);
+    }
+    prefix_length_encoder_.Put(prefix_lengths_.data(), batch_size);

Review Comment:
   My point is more that calling `DeltaLengthByteArrayEncoder::Put` once for each suffix is probably much less efficient than calling it once every K suffixes.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org